From 3a9f6fa6025a4e795c6fe0c31d0c07162e5549b6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Feb 2026 12:20:27 -0500 Subject: [PATCH 01/38] Adopt PR #3349's per-partition scan logic to CometNativeScan. Add DPP. --- .../scala/org/apache/comet/CometConf.scala | 7 - native/core/src/execution/planner.rs | 46 ++-- native/proto/src/proto/operator.proto | 29 ++- .../apache/comet/rules/CometExecRule.scala | 5 +- .../apache/comet/rules/CometScanRule.scala | 27 +- .../serde/operator/CometNativeScan.scala | 243 ++++++++++-------- .../comet/CometIcebergNativeScanExec.scala | 64 +---- .../spark/sql/comet/CometNativeScanExec.scala | 218 ++++++++++++---- .../apache/spark/sql/comet/operators.scala | 65 ++++- .../comet/shims/ShimSubqueryBroadcast.scala | 59 ++++- .../comet/shims/ShimSubqueryBroadcast.scala | 59 ++++- .../comet/shims/ShimSubqueryBroadcast.scala | 59 ++++- .../apache/comet/exec/CometExecSuite.scala | 141 +++++++++- .../ParquetReadFromFakeHadoopFsSuite.scala | 7 +- .../sql/comet/CometPlanStabilitySuite.scala | 1 - 15 files changed, 747 insertions(+), 283 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 522ccbc94c..c0b526e0a3 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -530,13 +530,6 @@ object CometConf extends ShimCometConf { .doubleConf .createWithDefault(1.0) - val COMET_DPP_FALLBACK_ENABLED: ConfigEntry[Boolean] = - conf("spark.comet.dppFallback.enabled") - .category(CATEGORY_EXEC) - .doc("Whether to fall back to Spark for queries that use DPP.") - .booleanConf - .createWithDefault(true) - val COMET_DEBUG_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.debug.enabled") .category(CATEGORY_EXEC) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 2c3d00a23b..0b74555f9b 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -965,20 +965,29 @@ impl PhysicalPlanner { )) } OpStruct::NativeScan(scan) => { - let data_schema = convert_spark_types_to_arrow_schema(scan.data_schema.as_slice()); + let common = scan + .common + .as_ref() + .ok_or_else(|| GeneralError("NativeScan missing common data".to_string()))?; + + let data_schema = + convert_spark_types_to_arrow_schema(common.data_schema.as_slice()); let required_schema: SchemaRef = - convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); + convert_spark_types_to_arrow_schema(common.required_schema.as_slice()); let partition_schema: SchemaRef = - convert_spark_types_to_arrow_schema(scan.partition_schema.as_slice()); - let projection_vector: Vec = scan + convert_spark_types_to_arrow_schema(common.partition_schema.as_slice()); + let projection_vector: Vec = common .projection_vector .iter() .map(|offset| *offset as usize) .collect(); - // Check if this partition has any files (bucketed scan with bucket pruning may have empty partitions) - let partition_files = &scan.file_partitions[self.partition as usize]; + // Get this partition's files (injected at execution time) + let partition_files = scan.file_partition.as_ref().ok_or_else(|| { + GeneralError("NativeScan missing file_partition data".to_string()) + })?; + // Check if this partition has any files (bucketed scan with bucket pruning may have empty partitions) if partition_files.partitioned_file.is_empty() { let empty_exec = Arc::new(EmptyExec::new(required_schema)); return Ok(( @@ -988,19 +997,19 @@ impl PhysicalPlanner { } // Convert the Spark expressions to Physical expressions - let data_filters: Result>, ExecutionError> = scan + let data_filters: Result>, ExecutionError> = common .data_filters .iter() .map(|expr| self.create_expr(expr, Arc::clone(&required_schema))) .collect(); - let default_values: Option> = if !scan + let default_values: Option> = if !common .default_values .is_empty() { // We have default values. Extract the two lists (same length) of values and // indexes in the schema, and then create a HashMap to use in the SchemaMapper. - let default_values: Result, DataFusionError> = scan + let default_values: Result, DataFusionError> = common .default_values .iter() .map(|expr| { @@ -1015,7 +1024,7 @@ impl PhysicalPlanner { }) .collect(); let default_values = default_values?; - let default_values_indexes: Vec = scan + let default_values_indexes: Vec = common .default_values_indexes .iter() .map(|offset| *offset as usize) @@ -1037,7 +1046,7 @@ impl PhysicalPlanner { .map(|f| f.file_path.clone()) .expect("partition should have files after empty check"); - let object_store_options: HashMap = scan + let object_store_options: HashMap = common .object_store_options .iter() .map(|(k, v)| (k.clone(), v.clone())) @@ -1048,10 +1057,7 @@ impl PhysicalPlanner { &object_store_options, )?; - // Comet serializes all partitions' PartitionedFiles, but we only want to read this - // Spark partition's PartitionedFiles - let files = - self.get_partitioned_files(&scan.file_partitions[self.partition as usize])?; + let files = self.get_partitioned_files(partition_files)?; let file_groups: Vec> = vec![files]; let partition_fields: Vec = partition_schema .fields() @@ -1060,7 +1066,7 @@ impl PhysicalPlanner { Field::new(field.name(), field.data_type().clone(), field.is_nullable()) }) .collect_vec(); - let scan = init_datasource_exec( + let datasource_exec = init_datasource_exec( required_schema, Some(data_schema), Some(partition_schema), @@ -1070,14 +1076,14 @@ impl PhysicalPlanner { Some(projection_vector), Some(data_filters?), default_values, - scan.session_timezone.as_str(), - scan.case_sensitive, + common.session_timezone.as_str(), + common.case_sensitive, self.session_ctx(), - scan.encryption_enabled, + common.encryption_enabled, )?; Ok(( vec![], - Arc::new(SparkPlan::new(spark_plan.plan_id, scan, vec![])), + Arc::new(SparkPlan::new(spark_plan.plan_id, datasource_exec, vec![])), )) } OpStruct::CsvScan(scan) => { diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 78f118e6db..2073f2701e 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -85,7 +85,8 @@ message Scan { bool arrow_ffi_safe = 3; } -message NativeScan { +// Common data shared across all partitions for NativeScan (sent once via commonBytes) +message NativeScanCommon { repeated spark.spark_expression.DataType fields = 1; // The source of the scan (e.g. file scan, broadcast exchange, shuffle, etc). This // is purely for informational purposes when viewing native query plans in @@ -95,20 +96,30 @@ message NativeScan { repeated SparkStructField data_schema = 4; repeated SparkStructField partition_schema = 5; repeated spark.spark_expression.Expr data_filters = 6; - repeated SparkFilePartition file_partitions = 7; - repeated int64 projection_vector = 8; - string session_timezone = 9; - repeated spark.spark_expression.Expr default_values = 10; - repeated int64 default_values_indexes = 11; - bool case_sensitive = 12; + repeated int64 projection_vector = 7; + string session_timezone = 8; + repeated spark.spark_expression.Expr default_values = 9; + repeated int64 default_values_indexes = 10; + bool case_sensitive = 11; // Options for configuring object stores such as AWS S3, GCS, etc. The key-value pairs are taken // from Hadoop configuration for compatibility with Hadoop FileSystem implementations of object // stores. // The configuration values have hadoop. or spark.hadoop. prefix trimmed. For instance, the // configuration value "spark.hadoop.fs.s3a.access.key" will be stored as "fs.s3a.access.key" in // the map. - map object_store_options = 13; - bool encryption_enabled = 14; + map object_store_options = 12; + bool encryption_enabled = 13; + + // Unique identifier for this scan, used to match planning data at execution time + string scan_id = 14; +} + +message NativeScan { + // Common data shared across partitions + NativeScanCommon common = 1; + + // This partition's files only (injected at execution time by NativePlanDataInjector) + SparkFilePartition file_partition = 2; } message CsvScan { diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 76e741e3bf..8ae32b3c30 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -180,9 +180,8 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { // spotless:on private def transform(plan: SparkPlan): SparkPlan = { def convertNode(op: SparkPlan): SparkPlan = op match { - // Fully native scan for V1 - case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => - convertToComet(scan, CometNativeScan).getOrElse(scan) + // CometNativeScanExec is created directly by CometScanRule and handles its own execution + // No conversion needed here - it passes through unchanged // Fully native Iceberg scan for V2 (iceberg-rust path) // Only handle scans with native metadata; SupportsComet scans fall through to isCometScan diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index ebb5217300..0a860e429f 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, DynamicPruningExpre import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.{sideBySide, ArrayBasedMapData, GenericArrayData, MetadataColumnHelper} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues -import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} +import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, InSubqueryExec, SparkPlan, SubqueryAdaptiveBroadcastExec} import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils @@ -50,6 +50,7 @@ import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflecti import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.{CometParquetScan, Native, SupportsComet} import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} +import org.apache.comet.serde.OperatorOuterClass import org.apache.comet.serde.operator.CometNativeScan import org.apache.comet.shims.{CometTypeShim, ShimFileFormat, ShimSubqueryBroadcast} @@ -140,10 +141,9 @@ case class CometScanRule(session: SparkSession) private def transformV1Scan(scanExec: FileSourceScanExec): SparkPlan = { - if (COMET_DPP_FALLBACK_ENABLED.get() && - scanExec.partitionFilters.exists(isDynamicPruningFilter)) { - return withInfo(scanExec, "Dynamic Partition Pruning is not supported") - } + // Check for DPP - only some scan implementations support it + val dppFilters = scanExec.partitionFilters.filter(isDynamicPruningFilter) + val hasDPP = dppFilters.nonEmpty scanExec.relation match { case r: HadoopFsRelation => @@ -170,13 +170,23 @@ case class CometScanRule(session: SparkSession) COMET_NATIVE_SCAN_IMPL.get() match { case SCAN_AUTO => // TODO add support for native_datafusion in the future + if (hasDPP) { + return withInfo(scanExec, "Dynamic Partition Pruning is not supported") + } nativeIcebergCompatScan(session, scanExec, r, hadoopConf) .getOrElse(scanExec) case SCAN_NATIVE_DATAFUSION => + // native_datafusion supports DPP nativeDataFusionScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) case SCAN_NATIVE_ICEBERG_COMPAT => + if (hasDPP) { + return withInfo(scanExec, "Dynamic Partition Pruning is not supported") + } nativeIcebergCompatScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) case SCAN_NATIVE_COMET => + if (hasDPP) { + return withInfo(scanExec, "Dynamic Partition Pruning is not supported") + } nativeCometScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) } @@ -213,7 +223,12 @@ case class CometScanRule(session: SparkSession) if (!isSchemaSupported(scanExec, SCAN_NATIVE_DATAFUSION, r)) { return None } - Some(CometScanExec(scanExec, session, SCAN_NATIVE_DATAFUSION)) + + // Create placeholder NativeScan operator + val builder = OperatorOuterClass.Operator.newBuilder() + CometNativeScan.convert(scanExec, builder).map { nativeOp => + CometNativeScanExec(nativeOp, scanExec) + } } private def nativeIcebergCompatScan( diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala index b7909b67cb..a5717a647d 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala @@ -25,25 +25,25 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PlanExpression} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues -import org.apache.spark.sql.comet.{CometNativeExec, CometNativeScanExec, CometScanExec} +import org.apache.spark.sql.comet.CometNativeScanExec import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.FilePartition import org.apache.spark.sql.internal.SQLConf -import org.apache.comet.{CometConf, ConfigEntry} +import org.apache.comet.CometConf import org.apache.comet.CometConf.COMET_EXEC_ENABLED import org.apache.comet.CometSparkSessionExtensions.{hasExplainInfo, withInfo} import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.CometParquetUtils -import org.apache.comet.serde.{CometOperatorSerde, Compatible, OperatorOuterClass, SupportLevel} import org.apache.comet.serde.ExprOuterClass.Expr +import org.apache.comet.serde.OperatorOuterClass import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType} /** * Validation and serde logic for `native_datafusion` scans. */ -object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { +object CometNativeScan extends Logging { /** Determine whether the scan is supported and tag the Spark plan with any fallback reasons */ def isSupported(scanExec: FileSourceScanExec): Boolean = { @@ -57,11 +57,6 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { withInfo(scanExec, s"Full native scan disabled because ${COMET_EXEC_ENABLED.key} disabled") } - // Native DataFusion doesn't support subqueries/dynamic pruning - if (scanExec.partitionFilters.exists(isDynamicPruningFilter)) { - withInfo(scanExec, "Native DataFusion scan does not support subqueries/dynamic pruning") - } - if (SQLConf.get.ignoreCorruptFiles || scanExec.relation.options .get("ignorecorruptfiles") // Spark sets this to lowercase. @@ -81,129 +76,151 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { !hasExplainInfo(scanExec) } - private def isDynamicPruningFilter(e: Expression): Boolean = + private[comet] def isDynamicPruningFilter(e: Expression): Boolean = e.exists(_.isInstanceOf[PlanExpression[_]]) - override def enabledConfig: Option[ConfigEntry[Boolean]] = None - - override def getSupportLevel(operator: CometScanExec): SupportLevel = { - // all checks happen in CometScanRule before ScanExec is converted to CometScanExec, so - // we always report compatible here because this serde object is for the converted CometScanExec - Compatible() - } - - override def convert( - scan: CometScanExec, - builder: Operator.Builder, - childOp: OperatorOuterClass.Operator*): Option[OperatorOuterClass.Operator] = { - val nativeScanBuilder = OperatorOuterClass.NativeScan.newBuilder() - nativeScanBuilder.setSource(scan.simpleStringWithNodeId()) + /** + * Convert FileSourceScanExec to a placeholder NativeScan operator. The actual partition data is + * populated at execution time by serializePartitions(). + */ + def convert( + scanExec: FileSourceScanExec, + builder: Operator.Builder): Option[OperatorOuterClass.Operator] = { - val scanTypes = scan.output.flatten { attr => + val scanTypes = scanExec.output.flatten { attr => serializeDataType(attr.dataType) } - if (scanTypes.length == scan.output.length) { - nativeScanBuilder.addAllFields(scanTypes.asJava) - - // Sink operators don't have children - builder.clearChildren() - - if (scan.conf.getConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED) && - CometConf.COMET_RESPECT_PARQUET_FILTER_PUSHDOWN.get(scan.conf)) { - - val dataFilters = new ListBuffer[Expr]() - for (filter <- scan.supportedDataFilters) { - exprToProto(filter, scan.output) match { - case Some(proto) => dataFilters += proto - case _ => - logWarning(s"Unsupported data filter $filter") - } - } - nativeScanBuilder.addAllDataFilters(dataFilters.asJava) - } - - val possibleDefaultValues = getExistenceDefaultValues(scan.requiredSchema) - if (possibleDefaultValues.exists(_ != null)) { - // Our schema has default values. Serialize two lists, one with the default values - // and another with the indexes in the schema so the native side can map missing - // columns to these default values. - val (defaultValues, indexes) = possibleDefaultValues.zipWithIndex - .filter { case (expr, _) => expr != null } - .map { case (expr, index) => - // ResolveDefaultColumnsUtil.getExistenceDefaultValues has evaluated these - // expressions and they should now just be literals. - (Literal(expr), index.toLong.asInstanceOf[java.lang.Long]) - } - .unzip - nativeScanBuilder.addAllDefaultValues( - defaultValues.flatMap(exprToProto(_, scan.output)).toIterable.asJava) - nativeScanBuilder.addAllDefaultValuesIndexes(indexes.toIterable.asJava) - } - - var firstPartition: Option[PartitionedFile] = None - val filePartitions = scan.getFilePartitions() - val filePartitionsProto = filePartitions.map { partition => - if (firstPartition.isEmpty) { - firstPartition = partition.files.headOption - } - partition2Proto(partition, scan.relation.partitionSchema) - } - nativeScanBuilder.addAllFilePartitions(filePartitionsProto.asJava) - - val partitionSchema = schema2Proto(scan.relation.partitionSchema.fields) - val requiredSchema = schema2Proto(scan.requiredSchema.fields) - val dataSchema = schema2Proto(scan.relation.dataSchema.fields) + if (scanTypes.length != scanExec.output.length) { + withInfo( + scanExec, + s"unsupported Comet operator: ${scanExec.nodeName}, due to unsupported data types above") + return None + } - val dataSchemaIndexes = scan.requiredSchema.fields.map(field => { - scan.relation.dataSchema.fieldIndex(field.name) - }) - val partitionSchemaIndexes = Array - .range( - scan.relation.dataSchema.fields.length, - scan.relation.dataSchema.length + scan.relation.partitionSchema.fields.length) + // Build placeholder NativeScan with just scan_id for matching at execution time. + // All other fields are populated by serializePartitions() at execution time. + val commonBuilder = OperatorOuterClass.NativeScanCommon.newBuilder() + commonBuilder.setScanId(getScanId(scanExec)) - val projectionVector = (dataSchemaIndexes ++ partitionSchemaIndexes).map(idx => - idx.toLong.asInstanceOf[java.lang.Long]) + val nativeScanBuilder = OperatorOuterClass.NativeScan.newBuilder() + nativeScanBuilder.setCommon(commonBuilder.build()) + // file_partition intentionally empty - will be populated at execution time - nativeScanBuilder.addAllProjectionVector(projectionVector.toIterable.asJava) + builder.clearChildren() + Some(builder.setNativeScan(nativeScanBuilder).build()) + } - // In `CometScanRule`, we ensure partitionSchema is supported. - assert(partitionSchema.length == scan.relation.partitionSchema.fields.length) + /** Unique identifier for this scan, used to match planning data at execution time. */ + def getScanId(scanExec: FileSourceScanExec): String = { + scanExec.relation.location.rootPaths.headOption + .map(_.toString) + .getOrElse(scanExec.simpleStringWithNodeId()) + } - nativeScanBuilder.addAllDataSchema(dataSchema.toIterable.asJava) - nativeScanBuilder.addAllRequiredSchema(requiredSchema.toIterable.asJava) - nativeScanBuilder.addAllPartitionSchema(partitionSchema.toIterable.asJava) - nativeScanBuilder.setSessionTimezone(scan.conf.getConfString("spark.sql.session.timeZone")) - nativeScanBuilder.setCaseSensitive(scan.conf.getConf[Boolean](SQLConf.CASE_SENSITIVE)) + /** + * Serializes partitions at execution time, after DPP filters have been resolved. + * + * @param exec + * The CometNativeScanExec with resolved DPP filters + * @param filePartitions + * The DPP-filtered file partitions (computed by exec using shim-compatible methods) + * @return + * Tuple of (commonBytes, perPartitionBytes) for native execution + */ + def serializePartitions( + exec: CometNativeScanExec, + filePartitions: Seq[FilePartition]): (Array[Byte], Array[Array[Byte]]) = { + val scanExec = exec.originalPlan + val relation = exec.relation + + val commonBuilder = OperatorOuterClass.NativeScanCommon.newBuilder() + commonBuilder.setSource(scanExec.simpleStringWithNodeId()) + commonBuilder.setScanId(exec.scanId) + + val scanTypes = exec.output.flatten { attr => + serializeDataType(attr.dataType) + } + commonBuilder.addAllFields(scanTypes.asJava) - // Collect S3/cloud storage configurations - val hadoopConf = scan.relation.sparkSession.sessionState - .newHadoopConfWithOptions(scan.relation.options) + // Filter out DPP filters from data filters - these are partition filters, not data filters + val supportedDataFilters = exec.dataFilters.filterNot(isDynamicPruningFilter) - nativeScanBuilder.setEncryptionEnabled(CometParquetUtils.encryptionEnabled(hadoopConf)) + val conf = relation.sparkSession.sessionState.conf + if (conf.getConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED) && + CometConf.COMET_RESPECT_PARQUET_FILTER_PUSHDOWN.get(conf)) { - firstPartition.foreach { partitionFile => - val objectStoreOptions = - NativeConfig.extractObjectStoreOptions(hadoopConf, partitionFile.pathUri) - objectStoreOptions.foreach { case (key, value) => - nativeScanBuilder.putObjectStoreOptions(key, value) + val dataFilters = new ListBuffer[Expr]() + for (filter <- supportedDataFilters) { + exprToProto(filter, exec.output) match { + case Some(proto) => dataFilters += proto + case _ => + logWarning(s"Unsupported data filter $filter") } } + commonBuilder.addAllDataFilters(dataFilters.asJava) + } - Some(builder.setNativeScan(nativeScanBuilder).build()) + val possibleDefaultValues = getExistenceDefaultValues(exec.requiredSchema) + if (possibleDefaultValues.exists(_ != null)) { + val (defaultValues, indexes) = possibleDefaultValues.zipWithIndex + .filter { case (expr, _) => expr != null } + .map { case (expr, index) => + (Literal(expr), index.toLong.asInstanceOf[java.lang.Long]) + } + .unzip + commonBuilder.addAllDefaultValues( + defaultValues.flatMap(exprToProto(_, exec.output)).toIterable.asJava) + commonBuilder.addAllDefaultValuesIndexes(indexes.toIterable.asJava) + } - } else { - // There are unsupported scan type - withInfo( - scan, - s"unsupported Comet operator: ${scan.nodeName}, due to unsupported data types above") - None + val partitionSchema = schema2Proto(relation.partitionSchema.fields) + val requiredSchema = schema2Proto(exec.requiredSchema.fields) + val dataSchema = schema2Proto(relation.dataSchema.fields) + + val dataSchemaIndexes = exec.requiredSchema.fields.map(field => { + relation.dataSchema.fieldIndex(field.name) + }) + val partitionSchemaIndexes = Array + .range( + relation.dataSchema.fields.length, + relation.dataSchema.length + relation.partitionSchema.fields.length) + + val projectionVector = (dataSchemaIndexes ++ partitionSchemaIndexes).map(idx => + idx.toLong.asInstanceOf[java.lang.Long]) + + commonBuilder.addAllProjectionVector(projectionVector.toIterable.asJava) + commonBuilder.addAllDataSchema(dataSchema.toIterable.asJava) + commonBuilder.addAllRequiredSchema(requiredSchema.toIterable.asJava) + commonBuilder.addAllPartitionSchema(partitionSchema.toIterable.asJava) + commonBuilder.setSessionTimezone(conf.getConfString("spark.sql.session.timeZone")) + commonBuilder.setCaseSensitive(conf.getConf[Boolean](SQLConf.CASE_SENSITIVE)) + + // Collect S3/cloud storage configurations + val hadoopConf = relation.sparkSession.sessionState + .newHadoopConfWithOptions(relation.options) + + commonBuilder.setEncryptionEnabled(CometParquetUtils.encryptionEnabled(hadoopConf)) + + filePartitions.headOption.flatMap(_.files.headOption).foreach { partitionFile => + val objectStoreOptions = + NativeConfig.extractObjectStoreOptions(hadoopConf, partitionFile.pathUri) + objectStoreOptions.foreach { case (key, value) => + commonBuilder.putObjectStoreOptions(key, value) + } } - } + val commonBytes = commonBuilder.build().toByteArray + + // Build per-partition data - each partition gets its own SparkFilePartition + val perPartitionBytes = filePartitions.map { partition => + val partitionProto = partition2Proto(partition, relation.partitionSchema) + val scanBuilder = OperatorOuterClass.NativeScan.newBuilder() + scanBuilder.setFilePartition(partitionProto) + // common is set at injection time, not here + scanBuilder.build().toByteArray + }.toArray - override def createExec(nativeOp: Operator, op: CometScanExec): CometNativeExec = { - CometNativeScanExec(nativeOp, op.wrapped, op.session) + (commonBytes, perPartitionBytes) } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index 36085b6329..d18b2ccf01 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -23,10 +23,10 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, DynamicPruningExpression, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, DynamicPruningExpression, SortOrder} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} -import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec} +import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -116,39 +116,12 @@ case class CometIcebergNativeScanExec( case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => e.plan match { case sab: SubqueryAdaptiveBroadcastExec => - // SubqueryAdaptiveBroadcastExec.executeCollect() throws, so we call - // child.executeCollect() directly. We use the index from SAB to find the - // right buildKey, then locate that key's column in child.output. - val rows = sab.child.executeCollect() - val indices = getSubqueryBroadcastIndices(sab) - - // SPARK-46946 changed index: Int to indices: Seq[Int] as a preparatory refactor - // for future features (Null Safe Equality DPP, multiple equality predicates). - // Currently indices always has one element. CometScanRule checks for multi-index - // DPP and falls back, so this assertion should never fail. - assert( - indices.length == 1, - s"Multi-index DPP not supported: indices=$indices. See SPARK-46946.") - val buildKeyIndex = indices.head - val buildKey = sab.buildKeys(buildKeyIndex) - - // Find column index in child.output by matching buildKey's exprId - val colIndex = buildKey match { - case attr: Attribute => - sab.child.output.indexWhere(_.exprId == attr.exprId) - // DPP may cast partition column to match join key type - case Cast(attr: Attribute, _, _, _) => - sab.child.output.indexWhere(_.exprId == attr.exprId) - case _ => buildKeyIndex - } - if (colIndex < 0) { - throw new IllegalStateException( - s"DPP build key '$buildKey' not found in ${sab.child.output.map(_.name)}") - } - - setInSubqueryResult(e, rows.map(_.get(colIndex, e.child.dataType))) - case _ => + resolveSubqueryAdaptiveBroadcast(sab, e) + case _: SubqueryBroadcastExec => e.updateResult() + case other => + throw new IllegalStateException( + s"Unexpected subquery plan type: ${other.getClass.getName}") } case _ => } @@ -156,29 +129,6 @@ case class CometIcebergNativeScanExec( CometIcebergNativeScan.serializePartitions(originalPlan, output, nativeIcebergScanMetadata) } - /** - * Sets InSubqueryExec's private result field via reflection. - * - * Reflection is required because: - * - SubqueryAdaptiveBroadcastExec.executeCollect() throws UnsupportedOperationException - * - InSubqueryExec has no public setter for result, only updateResult() which calls - * executeCollect() - * - We can't replace e.plan since it's a val - */ - private def setInSubqueryResult(e: InSubqueryExec, result: Array[_]): Unit = { - val fields = e.getClass.getDeclaredFields - // Field name is mangled by Scala compiler, e.g. "org$apache$...$InSubqueryExec$$result" - val resultField = fields - .find(f => f.getName.endsWith("$result") && !f.getName.contains("Broadcast")) - .getOrElse { - throw new IllegalStateException( - s"Cannot find 'result' field in ${e.getClass.getName}. " + - "Spark version may be incompatible with Comet's DPP implementation.") - } - resultField.setAccessible(true) - resultField.set(e, result) - } - def commonData: Array[Byte] = serializedPartitionData._1 def perPartitionData: Array[Array[Byte]] = serializedPartitionData._2 diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 841bc21aa2..13792d54ec 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -19,29 +19,32 @@ package org.apache.spark.sql.comet -import scala.reflect.ClassTag - +import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst._ +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} -import org.apache.spark.sql.comet.shims.ShimStreamSourceAwareSparkPlan +import org.apache.spark.sql.comet.shims.ShimCometScanExec import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.collection._ import com.google.common.base.Objects -import org.apache.comet.CometConf -import org.apache.comet.parquet.CometParquetFileFormat import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.operator.CometNativeScan +import org.apache.comet.shims.ShimSubqueryBroadcast /** * Comet fully native scan node for DataSource V1 that delegates to DataFusion's DataSourceExec. + * + * Wraps FileSourceScanExec directly (similar to CometIcebergNativeScanExec wrapping + * BatchScanExec). Supports Dynamic Partition Pruning (DPP) by deferring partition serialization + * to execution time. */ case class CometNativeScanExec( override val nativeOp: Operator, @@ -54,30 +57,163 @@ case class CometNativeScanExec( dataFilters: Seq[Expression], tableIdentifier: Option[TableIdentifier], disableBucketedScan: Boolean = false, - originalPlan: FileSourceScanExec, + @transient originalPlan: FileSourceScanExec, override val serializedPlanOpt: SerializedPlan) extends CometLeafExec with DataSourceScanExec - with ShimStreamSourceAwareSparkPlan { + with ShimCometScanExec + with ShimSubqueryBroadcast { override lazy val metadata: Map[String, String] = originalPlan.metadata + // Required by ShimCometScanExec for shim-compatible file splitting methods + override def wrapped: FileSourceScanExec = originalPlan + override val nodeName: String = s"CometNativeScan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" // exposed for testing lazy val bucketedScan: Boolean = originalPlan.bucketedScan && !disableBucketedScan + /** Unique identifier for this scan, used to match planning data at execution time. */ + def scanId: String = { + relation.location.rootPaths.headOption + .map(_.toString) + .getOrElse(originalPlan.simpleStringWithNodeId()) + } + + /** + * Prepare DPP subquery plans. Called by Spark's prepare() before doExecuteColumnar(). + */ + override protected def doPrepare(): Unit = { + partitionFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) => + e.plan.prepare() + case _ => + } + super.doPrepare() + } + + /** + * Lazy partition serialization - deferred until execution time for DPP support. + */ + @transient private lazy val serializedPartitionData: (Array[Byte], Array[Array[Byte]]) = { + // Wait for DPP subqueries to resolve before accessing partitions + partitionFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => + e.plan match { + case sab: SubqueryAdaptiveBroadcastExec => + resolveSubqueryAdaptiveBroadcast(sab, e) + case _: SubqueryBroadcastExec => + e.updateResult() + case other => + throw new IllegalStateException( + s"Unexpected subquery plan type: ${other.getClass.getName}") + } + case _ => + } + + val filePartitions = getFilePartitions() + CometNativeScan.serializePartitions(this, filePartitions) + } + + /** Get file partitions with DPP filtering applied. */ + private def getFilePartitions(): Seq[FilePartition] = { + val selectedPartitions = originalPlan.selectedPartitions + + val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) + val dynamicallySelectedPartitions = if (dynamicPartitionFilters.nonEmpty) { + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil) + selectedPartitions.filter(p => boundPredicate.eval(p.values)) + } else { + selectedPartitions + } + + createFilePartitionsForNonBucketedScan(dynamicallySelectedPartitions) + } + + private def isDynamicPruningFilter(e: Expression): Boolean = + e.exists(_.isInstanceOf[PlanExpression[_]]) + + private def createFilePartitionsForNonBucketedScan( + selectedPartitions: Array[PartitionDirectory]): Seq[FilePartition] = { + val maxSplitBytes = + FilePartition.maxSplitBytes(relation.sparkSession, selectedPartitions) + + // Filter files with bucket pruning if possible + val bucketingEnabled = relation.sparkSession.sessionState.conf.bucketingEnabled + val shouldProcess: Path => Boolean = optionalBucketSet match { + case Some(bucketSet) if bucketingEnabled => + filePath => BucketingUtils.getBucketId(filePath.getName).forall(bucketSet.get) + case _ => + _ => true + } + + val splitFilesList = selectedPartitions + .flatMap { partition => + partition.files.flatMap { file => + val filePath = file.getPath + + if (shouldProcess(filePath)) { + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, + relation.options, + filePath) && + file.getLen > maxSplitBytes + + splitFiles( + relation.sparkSession, + file, + filePath, + isSplitable, + maxSplitBytes, + partition.values) + } else { + Seq.empty + } + } + } + .sortBy(_.length)(implicitly[Ordering[Long]].reverse) + + FilePartition.getFilePartitions(relation.sparkSession, splitFilesList, maxSplitBytes) + } + + def commonData: Array[Byte] = serializedPartitionData._1 + def perPartitionData: Array[Array[Byte]] = serializedPartitionData._2 + override lazy val outputPartitioning: Partitioning = { if (bucketedScan) { originalPlan.outputPartitioning } else { - UnknownPartitioning(originalPlan.inputRDD.getNumPartitions) + UnknownPartitioning(perPartitionData.length) } } override lazy val outputOrdering: Seq[SortOrder] = originalPlan.outputOrdering + /** Executes using CometExecRDD - planning data is computed lazily on first access. */ + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val nativeMetrics = CometMetricNode.fromCometPlan(this) + val serializedPlan = CometExec.serializeNativePlan(nativeOp) + CometExecRDD( + sparkContext, + inputRDDs = Seq.empty, + commonByKey = Map(scanId -> commonData), + perPartitionByKey = Map(scanId -> perPartitionData), + serializedPlan = serializedPlan, + numPartitions = perPartitionData.length, + numOutputCols = output.length, + nativeMetrics = nativeMetrics, + subqueries = Seq.empty) + } + override def doCanonicalize(): CometNativeScanExec = { CometNativeScanExec( nativeOp, @@ -92,7 +228,7 @@ case class CometNativeScanExec( QueryPlan.normalizePredicates(dataFilters, output), None, disableBucketedScan, - originalPlan.doCanonicalize(), + null, // Don't need originalPlan for canonicalization SerializedPlan(None)) } @@ -111,7 +247,7 @@ case class CometNativeScanExec( override def hashCode(): Int = Objects.hashCode(originalPlan, serializedPlanOpt) override lazy val metrics: Map[String, SQLMetric] = - CometMetricNode.nativeScanMetrics(session.sparkContext) + CometMetricNode.nativeScanMetrics(sparkContext) /** * See [[org.apache.spark.sql.execution.DataSourceScanExec.inputRDDs]]. Only used for tests. @@ -120,49 +256,25 @@ case class CometNativeScanExec( } object CometNativeScanExec { - def apply( - nativeOp: Operator, - scanExec: FileSourceScanExec, - session: SparkSession): CometNativeScanExec = { - // TreeNode.mapProductIterator is protected method. - def mapProductIterator[B: ClassTag](product: Product, f: Any => B): Array[B] = { - val arr = Array.ofDim[B](product.productArity) - var i = 0 - while (i < arr.length) { - arr(i) = f(product.productElement(i)) - i += 1 - } - arr - } - - // Replacing the relation in FileSourceScanExec by `copy` seems causing some issues - // on other Spark distributions if FileSourceScanExec constructor is changed. - // Using `makeCopy` to avoid the issue. - // https://github.com/apache/arrow-datafusion-comet/issues/190 - def transform(arg: Any): AnyRef = arg match { - case _: HadoopFsRelation => - scanExec.relation.copy(fileFormat = - new CometParquetFileFormat(session, CometConf.SCAN_NATIVE_DATAFUSION))(session) - case other: AnyRef => other - case null => null - } - val newArgs = mapProductIterator(scanExec, transform) - val wrapped = scanExec.makeCopy(newArgs).asInstanceOf[FileSourceScanExec] - val batchScanExec = CometNativeScanExec( + /** + * Create CometNativeScanExec from a FileSourceScanExec. + */ + def apply(nativeOp: Operator, scanExec: FileSourceScanExec): CometNativeScanExec = { + val exec = CometNativeScanExec( nativeOp, - wrapped.relation, - wrapped.output, - wrapped.requiredSchema, - wrapped.partitionFilters, - wrapped.optionalBucketSet, - wrapped.optionalNumCoalescedBuckets, - wrapped.dataFilters, - wrapped.tableIdentifier, - wrapped.disableBucketedScan, - wrapped, + scanExec.relation, + scanExec.output, + scanExec.requiredSchema, + scanExec.partitionFilters, + scanExec.optionalBucketSet, + scanExec.optionalNumCoalescedBuckets, + scanExec.dataFilters, + scanExec.tableIdentifier, + scanExec.disableBucketedScan, + scanExec, SerializedPlan(None)) - scanExec.logicalLink.foreach(batchScanExec.setLogicalLink) - batchScanExec + scanExec.logicalLink.foreach(exec.setLogicalLink) + exec } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index eba74c9e25..9965aae633 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -81,10 +81,8 @@ private[comet] trait PlanDataInjector { private[comet] object PlanDataInjector { // Registry of injectors for different operator types - private val injectors: Seq[PlanDataInjector] = Seq( - IcebergPlanDataInjector - // Future: DeltaPlanDataInjector, HudiPlanDataInjector, etc. - ) + private val injectors: Seq[PlanDataInjector] = + Seq(IcebergPlanDataInjector, NativePlanDataInjector) /** * Injects planning data into an Operator tree by finding nodes that need injection and applying @@ -191,6 +189,60 @@ private[comet] object IcebergPlanDataInjector extends PlanDataInjector { } } +/** + * Injector for NativeScan operators. + */ +private[comet] object NativePlanDataInjector extends PlanDataInjector { + import java.nio.ByteBuffer + import java.util.{LinkedHashMap, Map => JMap} + + private final val maxCacheEntries = 16 + + // Cache parsed NativeScanCommon to avoid reparsing for tables with large numbers of partitions + private val commonCache = java.util.Collections.synchronizedMap( + new LinkedHashMap[ByteBuffer, OperatorOuterClass.NativeScanCommon](4, 0.75f, true) { + override def removeEldestEntry( + eldest: JMap.Entry[ByteBuffer, OperatorOuterClass.NativeScanCommon]): Boolean = { + size() > maxCacheEntries + } + }) + + override def canInject(op: Operator): Boolean = + op.hasNativeScan && + !op.getNativeScan.hasFilePartition && + op.getNativeScan.hasCommon + + override def getKey(op: Operator): Option[String] = + Some(op.getNativeScan.getCommon.getScanId) + + override def inject( + op: Operator, + commonBytes: Array[Byte], + partitionBytes: Array[Byte]): Operator = { + val scan = op.getNativeScan + + // Cache the parsed common data to avoid deserializing on every partition + val cacheKey = ByteBuffer.wrap(commonBytes) + val common = commonCache.synchronized { + Option(commonCache.get(cacheKey)).getOrElse { + val parsed = OperatorOuterClass.NativeScanCommon.parseFrom(commonBytes) + commonCache.put(cacheKey, parsed) + parsed + } + } + + val partitionOnly = OperatorOuterClass.NativeScan.parseFrom(partitionBytes) + + val scanBuilder = scan.toBuilder + scanBuilder.setCommon(common) + if (partitionOnly.hasFilePartition) { + scanBuilder.setFilePartition(partitionOnly.getFilePartition) + } + + op.toBuilder.setNativeScan(scanBuilder).build() + } +} + /** * A Comet physical operator */ @@ -589,6 +641,11 @@ abstract class CometNativeExec extends CometExec { Map(iceberg.metadataLocation -> iceberg.commonData), Map(iceberg.metadataLocation -> iceberg.perPartitionData)) + // Found a native scan with planning data + case native: CometNativeScanExec + if native.commonData.nonEmpty && native.perPartitionData.nonEmpty => + (Map(native.scanId -> native.commonData), Map(native.scanId -> native.perPartitionData)) + // Broadcast stages are boundaries - don't collect per-partition data from inside them. // After DPP filtering, broadcast scans may have different partition counts than the // probe side, causing ArrayIndexOutOfBoundsException in CometExecRDD.getPartitions. diff --git a/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala index 1ff0935041..6052bb5f61 100644 --- a/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala +++ b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -19,7 +19,8 @@ package org.apache.comet.shims -import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast} +import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec} trait ShimSubqueryBroadcast { @@ -30,4 +31,60 @@ trait ShimSubqueryBroadcast { def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { Seq(sab.index) } + + /** + * Resolves DPP values from SubqueryAdaptiveBroadcastExec and sets them on InSubqueryExec. + * + * SubqueryAdaptiveBroadcastExec.executeCollect() throws, so we call child.executeCollect() + * directly and use SAB's index to find the buildKey. + */ + def resolveSubqueryAdaptiveBroadcast( + sab: SubqueryAdaptiveBroadcastExec, + e: InSubqueryExec): Unit = { + val rows = sab.child.executeCollect() + val indices = getSubqueryBroadcastIndices(sab) + + // SPARK-46946 changed index: Int to indices: Seq[Int] for future features + // (null-safe equality, multiple predicates). Currently always one element. + assert( + indices.length == 1, + s"Multi-index DPP not supported: indices=$indices. See SPARK-46946.") + val buildKeyIndex = indices.head + val buildKey = sab.buildKeys(buildKeyIndex) + + val colIndex = buildKey match { + case attr: Attribute => + sab.child.output.indexWhere(_.exprId == attr.exprId) + // DPP may cast partition column to match join key type + case Cast(attr: Attribute, _, _, _) => + sab.child.output.indexWhere(_.exprId == attr.exprId) + case _ => buildKeyIndex + } + if (colIndex < 0) { + throw new IllegalStateException( + s"DPP build key '$buildKey' not found in ${sab.child.output.map(_.name)}") + } + + setInSubqueryResult(e, rows.map(_.get(colIndex, e.child.dataType))) + } + + /** + * Sets InSubqueryExec's private result field via reflection. + * + * Reflection is required because SubqueryAdaptiveBroadcastExec.executeCollect() throws, + * InSubqueryExec has no public setter for result, and we can't replace e.plan since it's a val. + */ + def setInSubqueryResult(e: InSubqueryExec, result: Array[_]): Unit = { + val fields = e.getClass.getDeclaredFields + // Field name is mangled by Scala compiler, e.g. "org$apache$...$InSubqueryExec$$result" + val resultField = fields + .find(f => f.getName.endsWith("$result") && !f.getName.contains("Broadcast")) + .getOrElse { + throw new IllegalStateException( + s"Cannot find 'result' field in ${e.getClass.getName}. " + + "Spark version may be incompatible with Comet's DPP implementation.") + } + resultField.setAccessible(true) + resultField.set(e, result) + } } diff --git a/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala index 1ff0935041..6052bb5f61 100644 --- a/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala +++ b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -19,7 +19,8 @@ package org.apache.comet.shims -import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast} +import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec} trait ShimSubqueryBroadcast { @@ -30,4 +31,60 @@ trait ShimSubqueryBroadcast { def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { Seq(sab.index) } + + /** + * Resolves DPP values from SubqueryAdaptiveBroadcastExec and sets them on InSubqueryExec. + * + * SubqueryAdaptiveBroadcastExec.executeCollect() throws, so we call child.executeCollect() + * directly and use SAB's index to find the buildKey. + */ + def resolveSubqueryAdaptiveBroadcast( + sab: SubqueryAdaptiveBroadcastExec, + e: InSubqueryExec): Unit = { + val rows = sab.child.executeCollect() + val indices = getSubqueryBroadcastIndices(sab) + + // SPARK-46946 changed index: Int to indices: Seq[Int] for future features + // (null-safe equality, multiple predicates). Currently always one element. + assert( + indices.length == 1, + s"Multi-index DPP not supported: indices=$indices. See SPARK-46946.") + val buildKeyIndex = indices.head + val buildKey = sab.buildKeys(buildKeyIndex) + + val colIndex = buildKey match { + case attr: Attribute => + sab.child.output.indexWhere(_.exprId == attr.exprId) + // DPP may cast partition column to match join key type + case Cast(attr: Attribute, _, _, _) => + sab.child.output.indexWhere(_.exprId == attr.exprId) + case _ => buildKeyIndex + } + if (colIndex < 0) { + throw new IllegalStateException( + s"DPP build key '$buildKey' not found in ${sab.child.output.map(_.name)}") + } + + setInSubqueryResult(e, rows.map(_.get(colIndex, e.child.dataType))) + } + + /** + * Sets InSubqueryExec's private result field via reflection. + * + * Reflection is required because SubqueryAdaptiveBroadcastExec.executeCollect() throws, + * InSubqueryExec has no public setter for result, and we can't replace e.plan since it's a val. + */ + def setInSubqueryResult(e: InSubqueryExec, result: Array[_]): Unit = { + val fields = e.getClass.getDeclaredFields + // Field name is mangled by Scala compiler, e.g. "org$apache$...$InSubqueryExec$$result" + val resultField = fields + .find(f => f.getName.endsWith("$result") && !f.getName.contains("Broadcast")) + .getOrElse { + throw new IllegalStateException( + s"Cannot find 'result' field in ${e.getClass.getName}. " + + "Spark version may be incompatible with Comet's DPP implementation.") + } + resultField.setAccessible(true) + resultField.set(e, result) + } } diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala index 417dfd46b7..e9f88f1d64 100644 --- a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -19,7 +19,8 @@ package org.apache.comet.shims -import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast} +import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec} trait ShimSubqueryBroadcast { @@ -30,4 +31,60 @@ trait ShimSubqueryBroadcast { def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { sab.indices } + + /** + * Resolves DPP values from SubqueryAdaptiveBroadcastExec and sets them on InSubqueryExec. + * + * SubqueryAdaptiveBroadcastExec.executeCollect() throws, so we call child.executeCollect() + * directly and use SAB's index to find the buildKey. + */ + def resolveSubqueryAdaptiveBroadcast( + sab: SubqueryAdaptiveBroadcastExec, + e: InSubqueryExec): Unit = { + val rows = sab.child.executeCollect() + val indices = getSubqueryBroadcastIndices(sab) + + // SPARK-46946 changed index: Int to indices: Seq[Int] for future features + // (null-safe equality, multiple predicates). Currently always one element. + assert( + indices.length == 1, + s"Multi-index DPP not supported: indices=$indices. See SPARK-46946.") + val buildKeyIndex = indices.head + val buildKey = sab.buildKeys(buildKeyIndex) + + val colIndex = buildKey match { + case attr: Attribute => + sab.child.output.indexWhere(_.exprId == attr.exprId) + // DPP may cast partition column to match join key type + case Cast(attr: Attribute, _, _, _) => + sab.child.output.indexWhere(_.exprId == attr.exprId) + case _ => buildKeyIndex + } + if (colIndex < 0) { + throw new IllegalStateException( + s"DPP build key '$buildKey' not found in ${sab.child.output.map(_.name)}") + } + + setInSubqueryResult(e, rows.map(_.get(colIndex, e.child.dataType))) + } + + /** + * Sets InSubqueryExec's private result field via reflection. + * + * Reflection is required because SubqueryAdaptiveBroadcastExec.executeCollect() throws, + * InSubqueryExec has no public setter for result, and we can't replace e.plan since it's a val. + */ + def setInSubqueryResult(e: InSubqueryExec, result: Array[_]): Unit = { + val fields = e.getClass.getDeclaredFields + // Field name is mangled by Scala compiler, e.g. "org$apache$...$InSubqueryExec$$result" + val resultField = fields + .find(f => f.getName.endsWith("$result") && !f.getName.contains("Broadcast")) + .getOrElse { + throw new IllegalStateException( + s"Cannot find 'result' field in ${e.getClass.getName}. " + + "Spark version may be incompatible with Comet's DPP implementation.") + } + resultField.setAccessible(true) + resultField.set(e, result) + } } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 6c9bdf6eba..daf90cd884 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -128,9 +128,7 @@ class CometExecSuite extends CometTestBase { // note that this test does not trigger DPP with v2 data source Seq("parquet").foreach { v1List => - withSQLConf( - SQLConf.USE_V1_SOURCE_LIST.key -> v1List, - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { spark.read.parquet(factPath).createOrReplaceTempView("dpp_fact") spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim") val df = @@ -146,6 +144,143 @@ class CometExecSuite extends CometTestBase { } } + test("DPP with native_datafusion scan - join with dynamic partition pruning") { + withTempDir { path => + val factPath = s"${path.getAbsolutePath}/fact_native" + val dimPath = s"${path.getAbsolutePath}/dim_native" + + withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "false") { + // Create fact table partitioned by date with 3 partitions + spark + .createDataFrame(Seq( + (1L, "a", java.sql.Date.valueOf("1970-01-01")), + (2L, "b", java.sql.Date.valueOf("1970-01-02")), + (3L, "c", java.sql.Date.valueOf("1970-01-02")), + (4L, "d", java.sql.Date.valueOf("1970-01-03")), + (5L, "e", java.sql.Date.valueOf("1970-01-01")), + (6L, "f", java.sql.Date.valueOf("1970-01-02")), + (7L, "g", java.sql.Date.valueOf("1970-01-03")), + (8L, "h", java.sql.Date.valueOf("1970-01-01")))) + .toDF("id", "data", "date") + .write + .partitionBy("date") + .parquet(factPath) + + // Create dimension table (small, to be broadcast) + spark + .createDataFrame(Seq((1L, java.sql.Date.valueOf("1970-01-02")))) + .toDF("id", "date") + .write + .parquet(dimPath) + } + + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, + // Prevent fact table from being broadcast (force dimension to be broadcast) + "spark.sql.autoBroadcastJoinThreshold" -> "1KB", + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true") { + + spark.read.parquet(factPath).createOrReplaceTempView("fact") + spark.read.parquet(dimPath).createOrReplaceTempView("dim") + + val query = + """SELECT /*+ BROADCAST(d) */ f.* FROM fact f + |JOIN dim d ON f.date = d.date AND d.id = 1 + |ORDER BY f.id""".stripMargin + + // Verify the plan contains dynamic pruning expression + val df = spark.sql(query) + val planStr = df.queryExecution.executedPlan.toString + assert( + planStr.contains("dynamicpruning"), + s"Expected dynamic pruning in plan but got:\n$planStr") + + // Verify native scan is used and DPP actually pruned partitions + val (_, cometPlan) = checkSparkAnswer(query) + val nativeScans = collect(cometPlan) { case s: CometNativeScanExec => s } + assert( + nativeScans.nonEmpty, + s"Expected CometNativeScanExec but found none. Plan:\n$cometPlan") + // DPP should reduce from 3 partition directories. File splitting may create + // multiple file partitions from one directory, so we check < 3 not == 1. + val numPartitions = nativeScans.head.perPartitionData.length + assert(numPartitions < 3, s"Expected DPP to prune partitions but got $numPartitions") + } + } + } + + test("DPP with native_datafusion scan - multiple partition columns") { + withTempDir { path => + val factPath = s"${path.getAbsolutePath}/fact_multi_part" + val dimPath = s"${path.getAbsolutePath}/dim_multi_part" + + withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "false") { + // Create fact table partitioned by TWO columns: region and category + val factData = Seq( + (1L, "data1", "US", "A"), + (2L, "data2", "US", "B"), + (3L, "data3", "US", "A"), + (4L, "data4", "EU", "A"), + (5L, "data5", "EU", "B"), + (6L, "data6", "EU", "A"), + (7L, "data7", "APAC", "A"), + (8L, "data8", "APAC", "B")) + spark + .createDataFrame(factData) + .toDF("id", "data", "region", "category") + .write + .partitionBy("region", "category") + .parquet(factPath) + + // Create dimension table + spark + .createDataFrame(Seq((1L, "US", "A"))) + .toDF("dim_id", "region", "category") + .write + .parquet(dimPath) + } + + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, + "spark.sql.autoBroadcastJoinThreshold" -> "1KB", + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true") { + + spark.read.parquet(factPath).createOrReplaceTempView("fact") + spark.read.parquet(dimPath).createOrReplaceTempView("dim") + + // Join on both partition columns - creates two DPP filters + val query = + """SELECT /*+ BROADCAST(d) */ f.* + |FROM fact f + |JOIN dim d ON f.region = d.region AND f.category = d.category + |WHERE d.dim_id = 1 + |ORDER BY f.id""".stripMargin + + // Verify plan has dynamic pruning expressions + val df = spark.sql(query) + val planStr = df.queryExecution.executedPlan.toString + assert( + planStr.contains("dynamicpruning"), + s"Expected dynamic pruning in plan but got:\n$planStr") + + // Verify native scan is used and DPP actually pruned partitions + val (_, cometPlan) = checkSparkAnswer(query) + val nativeScans = collect(cometPlan) { case s: CometNativeScanExec => s } + assert( + nativeScans.nonEmpty, + s"Expected CometNativeScanExec but found none. Plan:\n$cometPlan") + // With 6 partition combinations (3 regions x 2 categories), DPP should prune. + // We're filtering to region=US, category=A which is 1 partition. + val numPartitions = nativeScans.head.perPartitionData.length + assert(numPartitions < 6, s"Expected DPP to prune partitions but got $numPartitions") + } + } + } + test("ShuffleQueryStageExec could be direct child node of CometBroadcastExchangeExec") { withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val table = "src" diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala index f4a8b5ed82..859c53a9fb 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala @@ -66,11 +66,10 @@ class ParquetReadFromFakeHadoopFsSuite extends CometTestBase with AdaptiveSparkP p } assert(scans.size == 1) + // Access selected partitions through originalPlan (FileSourceScanExec) + val selectedPartitions = scans.head.originalPlan.selectedPartitions assert( - scans.head.nativeOp.getNativeScan - .getFilePartitions(0) - .getPartitionedFile(0) - .getFilePath + selectedPartitions.head.files.head.getPath.toString .startsWith(FakeHDFSFileSystem.PREFIX)) } diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index 69e52bd31a..fe4ce4cde9 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -218,7 +218,6 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_NATIVE_SCAN_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "false", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key -> "true", // as well as for v1.4/q9, v1.4/q44, v2.7.0/q6, v2.7.0/q64 From 9a3f7479ff9d78eed85a8998017005be5c0691a8 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Feb 2026 12:47:20 -0500 Subject: [PATCH 02/38] Fix encryption. --- .../spark/sql/comet/CometNativeScanExec.scala | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 13792d54ec..babac4804a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.comet import org.apache.hadoop.fs.Path +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions._ @@ -31,10 +32,12 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration import org.apache.spark.util.collection._ import com.google.common.base.Objects +import org.apache.comet.parquet.CometParquetUtils import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.operator.CometNativeScan import org.apache.comet.shims.ShimSubqueryBroadcast @@ -202,6 +205,19 @@ case class CometNativeScanExec( override def doExecuteColumnar(): RDD[ColumnarBatch] = { val nativeMetrics = CometMetricNode.fromCometPlan(this) val serializedPlan = CometExec.serializeNativePlan(nativeOp) + + // Handle encryption: broadcast hadoop conf if encryption is enabled + val hadoopConf = relation.sparkSession.sessionState + .newHadoopConfWithOptions(relation.options) + val (broadcastedHadoopConfForEncryption, encryptedFilePaths) = + if (CometParquetUtils.encryptionEnabled(hadoopConf)) { + val broadcastedConf = relation.sparkSession.sparkContext + .broadcast(new SerializableConfiguration(hadoopConf)) + (Some(broadcastedConf), relation.inputFiles.toSeq) + } else { + (None, Seq.empty[String]) + } + CometExecRDD( sparkContext, inputRDDs = Seq.empty, @@ -211,7 +227,9 @@ case class CometNativeScanExec( numPartitions = perPartitionData.length, numOutputCols = output.length, nativeMetrics = nativeMetrics, - subqueries = Seq.empty) + subqueries = Seq.empty, + broadcastedHadoopConfForEncryption = broadcastedHadoopConfForEncryption, + encryptedFilePaths = encryptedFilePaths) } override def doCanonicalize(): CometNativeScanExec = { From 9af450d87fb4dc7b243afafea9c5999b5beb9e96 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Feb 2026 13:01:34 -0500 Subject: [PATCH 03/38] Make format. --- .../scala/org/apache/spark/sql/comet/CometNativeScanExec.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index babac4804a..90568b60ff 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.comet import org.apache.hadoop.fs.Path -import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions._ From d9c4903a56be3a584f77fd5769e9f266ff305a26 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Feb 2026 13:16:59 -0500 Subject: [PATCH 04/38] Fix Spark 4 DPP API? --- .../spark/sql/comet/CometNativeScanExec.scala | 66 +------------------ .../sql/comet/shims/ShimCometScanExec.scala | 54 ++++++++++++++- .../sql/comet/shims/ShimCometScanExec.scala | 54 ++++++++++++++- .../sql/comet/shims/ShimCometScanExec.scala | 57 +++++++++++++++- 4 files changed, 162 insertions(+), 69 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 90568b60ff..8466a9a4eb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.comet -import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions._ @@ -121,70 +120,7 @@ case class CometNativeScanExec( /** Get file partitions with DPP filtering applied. */ private def getFilePartitions(): Seq[FilePartition] = { - val selectedPartitions = originalPlan.selectedPartitions - - val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) - val dynamicallySelectedPartitions = if (dynamicPartitionFilters.nonEmpty) { - val predicate = dynamicPartitionFilters.reduce(And) - val partitionColumns = relation.partitionSchema - val boundPredicate = Predicate.create( - predicate.transform { case a: AttributeReference => - val index = partitionColumns.indexWhere(a.name == _.name) - BoundReference(index, partitionColumns(index).dataType, nullable = true) - }, - Nil) - selectedPartitions.filter(p => boundPredicate.eval(p.values)) - } else { - selectedPartitions - } - - createFilePartitionsForNonBucketedScan(dynamicallySelectedPartitions) - } - - private def isDynamicPruningFilter(e: Expression): Boolean = - e.exists(_.isInstanceOf[PlanExpression[_]]) - - private def createFilePartitionsForNonBucketedScan( - selectedPartitions: Array[PartitionDirectory]): Seq[FilePartition] = { - val maxSplitBytes = - FilePartition.maxSplitBytes(relation.sparkSession, selectedPartitions) - - // Filter files with bucket pruning if possible - val bucketingEnabled = relation.sparkSession.sessionState.conf.bucketingEnabled - val shouldProcess: Path => Boolean = optionalBucketSet match { - case Some(bucketSet) if bucketingEnabled => - filePath => BucketingUtils.getBucketId(filePath.getName).forall(bucketSet.get) - case _ => - _ => true - } - - val splitFilesList = selectedPartitions - .flatMap { partition => - partition.files.flatMap { file => - val filePath = file.getPath - - if (shouldProcess(filePath)) { - val isSplitable = relation.fileFormat.isSplitable( - relation.sparkSession, - relation.options, - filePath) && - file.getLen > maxSplitBytes - - splitFiles( - relation.sparkSession, - file, - filePath, - isSplitable, - maxSplitBytes, - partition.values) - } else { - Seq.empty - } - } - } - .sortBy(_.length)(implicitly[Ordering[Long]].reverse) - - FilePartition.getFilePartitions(relation.sparkSession, splitFilesList, maxSplitBytes) + getDppFilteredFilePartitions(relation, partitionFilters, originalPlan.selectedPartitions) } def commonData: Array[Byte] = serializedPartitionData._1 diff --git a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 7d8ba9f0f8..c3e1a41261 100644 --- a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -22,7 +22,7 @@ package org.apache.spark.sql.comet.shims import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions @@ -37,6 +37,58 @@ trait ShimCometScanExec { lazy val fileConstantMetadataColumns: Seq[AttributeReference] = wrapped.fileConstantMetadataColumns + /** + * Returns file partitions after applying DPP filtering. In Spark 3.x, filters + * Array[PartitionDirectory] manually. + * + * Based on FileSourceScanExec.dynamicallySelectedPartitions and + * FileSourceScanExec.createNonBucketedReadRDD. + */ + protected def getDppFilteredFilePartitions( + relation: HadoopFsRelation, + partitionFilters: Seq[Expression], + selectedPartitions: Array[PartitionDirectory]): Seq[FilePartition] = { + val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) + val filteredPartitions = if (dynamicPartitionFilters.nonEmpty) { + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil) + selectedPartitions.filter(p => boundPredicate.eval(p.values)) + } else { + selectedPartitions + } + + val maxSplitBytes = FilePartition.maxSplitBytes(relation.sparkSession, filteredPartitions) + val splitFilesList = filteredPartitions + .flatMap { partition => + partition.files.flatMap { file => + val filePath = file.getPath + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, + relation.options, + filePath) && file.getLen > maxSplitBytes + splitFiles( + relation.sparkSession, + file, + filePath, + isSplitable, + maxSplitBytes, + partition.values) + } + } + .sortBy(_.length)(implicitly[Ordering[Long]].reverse) + + FilePartition.getFilePartitions(relation.sparkSession, splitFilesList, maxSplitBytes) + } + + private def isDynamicPruningFilter(e: Expression): Boolean = + e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) + protected def newFileScanRDD( fsRelation: HadoopFsRelation, readFunction: PartitionedFile => Iterator[InternalRow], diff --git a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 12be0cc53f..33b1b7b18d 100644 --- a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SPARK_VERSION_SHORT import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions @@ -39,6 +39,58 @@ trait ShimCometScanExec { lazy val fileConstantMetadataColumns: Seq[AttributeReference] = wrapped.fileConstantMetadataColumns + /** + * Returns file partitions after applying DPP filtering. In Spark 3.x, filters + * Array[PartitionDirectory] manually. + * + * Based on FileSourceScanExec.dynamicallySelectedPartitions and + * FileSourceScanExec.createNonBucketedReadRDD. + */ + protected def getDppFilteredFilePartitions( + relation: HadoopFsRelation, + partitionFilters: Seq[Expression], + selectedPartitions: Array[PartitionDirectory]): Seq[FilePartition] = { + val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) + val filteredPartitions = if (dynamicPartitionFilters.nonEmpty) { + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil) + selectedPartitions.filter(p => boundPredicate.eval(p.values)) + } else { + selectedPartitions + } + + val maxSplitBytes = FilePartition.maxSplitBytes(relation.sparkSession, filteredPartitions) + val splitFilesList = filteredPartitions + .flatMap { partition => + partition.files.flatMap { file => + val filePath = file.getPath + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, + relation.options, + filePath) && file.getLen > maxSplitBytes + splitFiles( + relation.sparkSession, + file, + filePath, + isSplitable, + maxSplitBytes, + partition.values) + } + } + .sortBy(_.length)(implicitly[Ordering[Long]].reverse) + + FilePartition.getFilePartitions(relation.sparkSession, splitFilesList, maxSplitBytes) + } + + private def isDynamicPruningFilter(e: Expression): Boolean = + e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) + def isSparkVersionAtLeast355: Boolean = { VersionUtils.majorMinorPatchVersion(SPARK_VERSION_SHORT) match { case Some((major, minor, patch)) => (major, minor, patch) >= (3, 5, 5) diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 3d9b963a93..d6cef721eb 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -22,8 +22,8 @@ package org.apache.spark.sql.comet.shims import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, FileSourceConstantMetadataAttribute, Literal} -import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil, ScalarSubquery} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BasePredicate, BoundReference, Expression, FileSourceConstantMetadataAttribute, Literal, Predicate} +import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil, ScalarSubquery, ScanFileListing} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.sources.Filter @@ -35,6 +35,59 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { lazy val fileConstantMetadataColumns: Seq[AttributeReference] = wrapped.fileConstantMetadataColumns + /** + * Returns file partitions after applying DPP filtering. In Spark 4.0, uses + * ScanFileListing.filterAndPruneFiles. + * + * Based on FileSourceScanExec.dynamicallySelectedPartitions and + * FileSourceScanLike.createScanFileSplitsForNonBucketedScan. + */ + protected def getDppFilteredFilePartitions( + relation: HadoopFsRelation, + partitionFilters: Seq[Expression], + selectedPartitions: ScanFileListing): Seq[FilePartition] = { + val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) + val filteredListing = if (dynamicPartitionFilters.nonEmpty) { + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil) + selectedPartitions.filterAndPruneFiles(boundPredicate, Seq.empty) + } else { + selectedPartitions + } + + val maxSplitBytes = FilePartition.maxSplitBytes(relation.sparkSession, filteredListing) + val splitFiles = filteredListing.filePartitionIterator + .flatMap { partition => + partition.files.flatMap { file => + val filePath = file.getPath + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, + relation.options, + filePath) && file.getLen > maxSplitBytes + splitFiles( + relation.sparkSession, + file, + filePath, + isSplitable, + maxSplitBytes, + partition.values) + } + } + .toSeq + .sortBy(_.length)(implicitly[Ordering[Long]].reverse) + + FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) + } + + private def isDynamicPruningFilter(e: Expression): Boolean = + e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) + protected def newFileScanRDD( fsRelation: HadoopFsRelation, readFunction: PartitionedFile => Iterator[InternalRow], From f57222086729c6c2ca7b0dbb0a3691f6d05c6b30 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Feb 2026 16:29:51 -0500 Subject: [PATCH 05/38] New plans. --- .../sql/comet/shims/ShimCometScanExec.scala | 4 +- .../q1.native_datafusion/extended.txt | 111 +- .../q1.native_iceberg_compat/extended.txt | 106 +- .../q10.native_datafusion/extended.txt | 72 +- .../q10.native_iceberg_compat/extended.txt | 62 +- .../q11.native_datafusion/extended.txt | 203 +- .../q11.native_iceberg_compat/extended.txt | 195 +- .../q12.native_datafusion/extended.txt | 54 +- .../q12.native_iceberg_compat/extended.txt | 54 +- .../q13.native_datafusion/extended.txt | 88 +- .../q13.native_iceberg_compat/extended.txt | 88 +- .../q14a.native_datafusion/extended.txt | 1099 ++-- .../q14a.native_iceberg_compat/extended.txt | 1014 +-- .../q14b.native_datafusion/extended.txt | 796 ++- .../q14b.native_iceberg_compat/extended.txt | 741 ++- .../q15.native_datafusion/extended.txt | 66 +- .../q15.native_iceberg_compat/extended.txt | 66 +- .../q17.native_datafusion/extended.txt | 135 +- .../q17.native_iceberg_compat/extended.txt | 130 +- .../q18.native_datafusion/extended.txt | 107 +- .../q18.native_iceberg_compat/extended.txt | 107 +- .../q19.native_datafusion/extended.txt | 2 +- .../q20.native_datafusion/extended.txt | 54 +- .../q20.native_iceberg_compat/extended.txt | 54 +- .../q21.native_datafusion/extended.txt | 64 +- .../q21.native_iceberg_compat/extended.txt | 64 +- .../q22.native_datafusion/extended.txt | 68 +- .../q22.native_iceberg_compat/extended.txt | 68 +- .../q23a.native_datafusion/extended.txt | 203 +- .../q23a.native_iceberg_compat/extended.txt | 198 +- .../q23b.native_datafusion/extended.txt | 207 +- .../q23b.native_iceberg_compat/extended.txt | 202 +- .../q25.native_datafusion/extended.txt | 135 +- .../q25.native_iceberg_compat/extended.txt | 130 +- .../q26.native_datafusion/extended.txt | 81 +- .../q26.native_iceberg_compat/extended.txt | 81 +- .../q27.native_datafusion/extended.txt | 83 +- .../q27.native_iceberg_compat/extended.txt | 83 +- .../q29.native_datafusion/extended.txt | 140 +- .../q29.native_iceberg_compat/extended.txt | 140 +- .../q30.native_datafusion/extended.txt | 137 +- .../q30.native_iceberg_compat/extended.txt | 132 +- .../q31.native_datafusion/extended.txt | 282 +- .../q31.native_iceberg_compat/extended.txt | 270 +- .../q32.native_datafusion/extended.txt | 93 +- .../q32.native_iceberg_compat/extended.txt | 88 +- .../q33.native_datafusion/extended.txt | 217 +- .../q33.native_iceberg_compat/extended.txt | 207 +- .../q34.native_datafusion/extended.txt | 82 +- .../q34.native_iceberg_compat/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35.native_iceberg_compat/extended.txt | 62 +- .../q36.native_datafusion/extended.txt | 69 +- .../q36.native_iceberg_compat/extended.txt | 69 +- .../q37.native_datafusion/extended.txt | 64 +- .../q37.native_iceberg_compat/extended.txt | 64 +- .../q38.native_datafusion/extended.txt | 133 +- .../q38.native_iceberg_compat/extended.txt | 123 +- .../q39a.native_datafusion/extended.txt | 134 +- .../q39a.native_iceberg_compat/extended.txt | 134 +- .../q39b.native_datafusion/extended.txt | 134 +- .../q39b.native_iceberg_compat/extended.txt | 134 +- .../q4.native_datafusion/extended.txt | 299 +- .../q4.native_iceberg_compat/extended.txt | 283 +- .../q40.native_datafusion/extended.txt | 19 +- .../q40.native_iceberg_compat/extended.txt | 19 +- .../q45.native_datafusion/extended.txt | 66 +- .../q45.native_iceberg_compat/extended.txt | 66 +- .../q46.native_datafusion/extended.txt | 99 +- .../q46.native_iceberg_compat/extended.txt | 99 +- .../q47.native_datafusion/extended.txt | 179 +- .../q47.native_iceberg_compat/extended.txt | 179 +- .../q48.native_datafusion/extended.txt | 77 +- .../q48.native_iceberg_compat/extended.txt | 77 +- .../q49.native_datafusion/extended.txt | 154 +- .../q49.native_iceberg_compat/extended.txt | 144 +- .../q5.native_datafusion/extended.txt | 219 +- .../q5.native_iceberg_compat/extended.txt | 194 +- .../q50.native_datafusion/extended.txt | 77 +- .../q50.native_iceberg_compat/extended.txt | 77 +- .../q51.native_datafusion/extended.txt | 75 +- .../q51.native_iceberg_compat/extended.txt | 70 +- .../q53.native_datafusion/extended.txt | 65 +- .../q53.native_iceberg_compat/extended.txt | 65 +- .../q54.native_datafusion/extended.txt | 210 +- .../q54.native_iceberg_compat/extended.txt | 205 +- .../q56.native_datafusion/extended.txt | 223 +- .../q56.native_iceberg_compat/extended.txt | 213 +- .../q57.native_datafusion/extended.txt | 179 +- .../q57.native_iceberg_compat/extended.txt | 179 +- .../q58.native_datafusion/extended.txt | 263 +- .../q58.native_iceberg_compat/extended.txt | 233 +- .../q6.native_datafusion/extended.txt | 123 +- .../q6.native_iceberg_compat/extended.txt | 123 +- .../q60.native_datafusion/extended.txt | 223 +- .../q60.native_iceberg_compat/extended.txt | 213 +- .../q61.native_datafusion/extended.txt | 188 +- .../q61.native_iceberg_compat/extended.txt | 183 +- .../q62.native_datafusion/extended.txt | 2 +- .../q63.native_datafusion/extended.txt | 65 +- .../q63.native_iceberg_compat/extended.txt | 65 +- .../q64.native_datafusion/extended.txt | 70 +- .../q64.native_iceberg_compat/extended.txt | 62 +- .../q65.native_datafusion/extended.txt | 115 +- .../q65.native_iceberg_compat/extended.txt | 110 +- .../q66.native_datafusion/extended.txt | 154 +- .../q66.native_iceberg_compat/extended.txt | 150 +- .../q67.native_datafusion/extended.txt | 72 +- .../q67.native_iceberg_compat/extended.txt | 72 +- .../q68.native_datafusion/extended.txt | 99 +- .../q68.native_iceberg_compat/extended.txt | 99 +- .../q69.native_datafusion/extended.txt | 72 +- .../q69.native_iceberg_compat/extended.txt | 62 +- .../q7.native_datafusion/extended.txt | 81 +- .../q7.native_iceberg_compat/extended.txt | 81 +- .../q70.native_datafusion/extended.txt | 77 +- .../q70.native_iceberg_compat/extended.txt | 72 +- .../q71.native_datafusion/extended.txt | 119 +- .../q71.native_iceberg_compat/extended.txt | 109 +- .../q72.native_datafusion/extended.txt | 126 +- .../q72.native_iceberg_compat/extended.txt | 126 +- .../q73.native_datafusion/extended.txt | 82 +- .../q73.native_iceberg_compat/extended.txt | 82 +- .../q74.native_datafusion/extended.txt | 201 +- .../q74.native_iceberg_compat/extended.txt | 193 +- .../q75.native_datafusion/extended.txt | 240 +- .../q75.native_iceberg_compat/extended.txt | 216 +- .../q77.native_datafusion/extended.txt | 257 +- .../q77.native_iceberg_compat/extended.txt | 232 +- .../q78.native_datafusion/extended.txt | 49 +- .../q78.native_iceberg_compat/extended.txt | 37 +- .../q79.native_datafusion/extended.txt | 75 +- .../q79.native_iceberg_compat/extended.txt | 75 +- .../q8.native_datafusion/extended.txt | 106 +- .../q8.native_iceberg_compat/extended.txt | 106 +- .../q80.native_datafusion/extended.txt | 57 +- .../q80.native_iceberg_compat/extended.txt | 39 +- .../q81.native_datafusion/extended.txt | 137 +- .../q81.native_iceberg_compat/extended.txt | 132 +- .../q82.native_datafusion/extended.txt | 64 +- .../q82.native_iceberg_compat/extended.txt | 64 +- .../q83.native_datafusion/extended.txt | 247 +- .../q83.native_iceberg_compat/extended.txt | 219 +- .../q85.native_datafusion/extended.txt | 118 +- .../q85.native_iceberg_compat/extended.txt | 118 +- .../q86.native_datafusion/extended.txt | 56 +- .../q86.native_iceberg_compat/extended.txt | 56 +- .../q87.native_datafusion/extended.txt | 133 +- .../q87.native_iceberg_compat/extended.txt | 123 +- .../q89.native_datafusion/extended.txt | 65 +- .../q89.native_iceberg_compat/extended.txt | 65 +- .../q91.native_datafusion/extended.txt | 104 +- .../q91.native_iceberg_compat/extended.txt | 104 +- .../q92.native_datafusion/extended.txt | 93 +- .../q92.native_iceberg_compat/extended.txt | 88 +- .../q97.native_datafusion/extended.txt | 61 +- .../q97.native_iceberg_compat/extended.txt | 56 +- .../q98.native_datafusion/extended.txt | 54 +- .../q98.native_iceberg_compat/extended.txt | 54 +- .../q99.native_datafusion/extended.txt | 2 +- .../q1.native_datafusion/extended.txt | 111 +- .../q1.native_iceberg_compat/extended.txt | 106 +- .../q10.native_datafusion/extended.txt | 72 +- .../q10.native_iceberg_compat/extended.txt | 62 +- .../q11.native_datafusion/extended.txt | 203 +- .../q11.native_iceberg_compat/extended.txt | 195 +- .../q12.native_datafusion/extended.txt | 54 +- .../q12.native_iceberg_compat/extended.txt | 54 +- .../q13.native_datafusion/extended.txt | 88 +- .../q13.native_iceberg_compat/extended.txt | 88 +- .../q14a.native_datafusion/extended.txt | 1099 ++-- .../q14a.native_iceberg_compat/extended.txt | 1014 +-- .../q14b.native_datafusion/extended.txt | 820 +-- .../q14b.native_iceberg_compat/extended.txt | 749 ++- .../q15.native_datafusion/extended.txt | 66 +- .../q15.native_iceberg_compat/extended.txt | 66 +- .../q17.native_datafusion/extended.txt | 135 +- .../q17.native_iceberg_compat/extended.txt | 130 +- .../q18.native_datafusion/extended.txt | 107 +- .../q18.native_iceberg_compat/extended.txt | 107 +- .../q19.native_datafusion/extended.txt | 2 +- .../q20.native_datafusion/extended.txt | 54 +- .../q20.native_iceberg_compat/extended.txt | 54 +- .../q21.native_datafusion/extended.txt | 64 +- .../q21.native_iceberg_compat/extended.txt | 64 +- .../q22.native_datafusion/extended.txt | 68 +- .../q22.native_iceberg_compat/extended.txt | 68 +- .../q23a.native_datafusion/extended.txt | 203 +- .../q23a.native_iceberg_compat/extended.txt | 198 +- .../q23b.native_datafusion/extended.txt | 207 +- .../q23b.native_iceberg_compat/extended.txt | 202 +- .../q25.native_datafusion/extended.txt | 135 +- .../q25.native_iceberg_compat/extended.txt | 130 +- .../q26.native_datafusion/extended.txt | 81 +- .../q26.native_iceberg_compat/extended.txt | 81 +- .../q27.native_datafusion/extended.txt | 83 +- .../q27.native_iceberg_compat/extended.txt | 83 +- .../q29.native_datafusion/extended.txt | 140 +- .../q29.native_iceberg_compat/extended.txt | 140 +- .../q30.native_datafusion/extended.txt | 137 +- .../q30.native_iceberg_compat/extended.txt | 132 +- .../q31.native_datafusion/extended.txt | 282 +- .../q31.native_iceberg_compat/extended.txt | 270 +- .../q32.native_datafusion/extended.txt | 93 +- .../q32.native_iceberg_compat/extended.txt | 88 +- .../q33.native_datafusion/extended.txt | 217 +- .../q33.native_iceberg_compat/extended.txt | 207 +- .../q34.native_datafusion/extended.txt | 82 +- .../q34.native_iceberg_compat/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35.native_iceberg_compat/extended.txt | 62 +- .../q36.native_datafusion/extended.txt | 69 +- .../q36.native_iceberg_compat/extended.txt | 69 +- .../q37.native_datafusion/extended.txt | 64 +- .../q37.native_iceberg_compat/extended.txt | 64 +- .../q38.native_datafusion/extended.txt | 133 +- .../q38.native_iceberg_compat/extended.txt | 123 +- .../q39a.native_datafusion/extended.txt | 134 +- .../q39a.native_iceberg_compat/extended.txt | 134 +- .../q39b.native_datafusion/extended.txt | 134 +- .../q39b.native_iceberg_compat/extended.txt | 134 +- .../q4.native_datafusion/extended.txt | 299 +- .../q4.native_iceberg_compat/extended.txt | 283 +- .../q40.native_datafusion/extended.txt | 19 +- .../q40.native_iceberg_compat/extended.txt | 19 +- .../q45.native_datafusion/extended.txt | 66 +- .../q45.native_iceberg_compat/extended.txt | 66 +- .../q46.native_datafusion/extended.txt | 99 +- .../q46.native_iceberg_compat/extended.txt | 99 +- .../q47.native_datafusion/extended.txt | 179 +- .../q47.native_iceberg_compat/extended.txt | 179 +- .../q48.native_datafusion/extended.txt | 77 +- .../q48.native_iceberg_compat/extended.txt | 77 +- .../q49.native_datafusion/extended.txt | 154 +- .../q49.native_iceberg_compat/extended.txt | 144 +- .../q5.native_datafusion/extended.txt | 219 +- .../q5.native_iceberg_compat/extended.txt | 194 +- .../q50.native_datafusion/extended.txt | 77 +- .../q50.native_iceberg_compat/extended.txt | 77 +- .../q51.native_datafusion/extended.txt | 75 +- .../q51.native_iceberg_compat/extended.txt | 70 +- .../q53.native_datafusion/extended.txt | 65 +- .../q53.native_iceberg_compat/extended.txt | 65 +- .../q54.native_datafusion/extended.txt | 244 +- .../q54.native_iceberg_compat/extended.txt | 213 +- .../q56.native_datafusion/extended.txt | 223 +- .../q56.native_iceberg_compat/extended.txt | 213 +- .../q57.native_datafusion/extended.txt | 179 +- .../q57.native_iceberg_compat/extended.txt | 179 +- .../q58.native_datafusion/extended.txt | 285 +- .../q58.native_iceberg_compat/extended.txt | 241 +- .../q6.native_datafusion/extended.txt | 141 +- .../q6.native_iceberg_compat/extended.txt | 127 +- .../q60.native_datafusion/extended.txt | 223 +- .../q60.native_iceberg_compat/extended.txt | 213 +- .../q61.native_datafusion/extended.txt | 188 +- .../q61.native_iceberg_compat/extended.txt | 183 +- .../q62.native_datafusion/extended.txt | 2 +- .../q63.native_datafusion/extended.txt | 65 +- .../q63.native_iceberg_compat/extended.txt | 65 +- .../q64.native_datafusion/extended.txt | 70 +- .../q64.native_iceberg_compat/extended.txt | 62 +- .../q65.native_datafusion/extended.txt | 115 +- .../q65.native_iceberg_compat/extended.txt | 110 +- .../q66.native_datafusion/extended.txt | 154 +- .../q66.native_iceberg_compat/extended.txt | 150 +- .../q67.native_datafusion/extended.txt | 72 +- .../q67.native_iceberg_compat/extended.txt | 72 +- .../q68.native_datafusion/extended.txt | 99 +- .../q68.native_iceberg_compat/extended.txt | 99 +- .../q69.native_datafusion/extended.txt | 72 +- .../q69.native_iceberg_compat/extended.txt | 62 +- .../q7.native_datafusion/extended.txt | 81 +- .../q7.native_iceberg_compat/extended.txt | 81 +- .../q70.native_datafusion/extended.txt | 77 +- .../q70.native_iceberg_compat/extended.txt | 72 +- .../q71.native_datafusion/extended.txt | 119 +- .../q71.native_iceberg_compat/extended.txt | 109 +- .../q72.native_datafusion/extended.txt | 126 +- .../q72.native_iceberg_compat/extended.txt | 126 +- .../q73.native_datafusion/extended.txt | 82 +- .../q73.native_iceberg_compat/extended.txt | 82 +- .../q74.native_datafusion/extended.txt | 201 +- .../q74.native_iceberg_compat/extended.txt | 193 +- .../q75.native_datafusion/extended.txt | 240 +- .../q75.native_iceberg_compat/extended.txt | 216 +- .../q77.native_datafusion/extended.txt | 257 +- .../q77.native_iceberg_compat/extended.txt | 232 +- .../q78.native_datafusion/extended.txt | 49 +- .../q78.native_iceberg_compat/extended.txt | 37 +- .../q79.native_datafusion/extended.txt | 75 +- .../q79.native_iceberg_compat/extended.txt | 75 +- .../q8.native_datafusion/extended.txt | 106 +- .../q8.native_iceberg_compat/extended.txt | 106 +- .../q80.native_datafusion/extended.txt | 57 +- .../q80.native_iceberg_compat/extended.txt | 39 +- .../q81.native_datafusion/extended.txt | 137 +- .../q81.native_iceberg_compat/extended.txt | 132 +- .../q82.native_datafusion/extended.txt | 64 +- .../q82.native_iceberg_compat/extended.txt | 64 +- .../q83.ansi.native_datafusion/extended.txt | 247 +- .../extended.txt | 219 +- .../q85.native_datafusion/extended.txt | 118 +- .../q85.native_iceberg_compat/extended.txt | 118 +- .../q86.native_datafusion/extended.txt | 56 +- .../q86.native_iceberg_compat/extended.txt | 56 +- .../q87.native_datafusion/extended.txt | 133 +- .../q87.native_iceberg_compat/extended.txt | 123 +- .../q89.native_datafusion/extended.txt | 65 +- .../q89.native_iceberg_compat/extended.txt | 65 +- .../q91.native_datafusion/extended.txt | 104 +- .../q91.native_iceberg_compat/extended.txt | 104 +- .../q92.native_datafusion/extended.txt | 93 +- .../q92.native_iceberg_compat/extended.txt | 88 +- .../q97.native_datafusion/extended.txt | 61 +- .../q97.native_iceberg_compat/extended.txt | 56 +- .../q98.native_datafusion/extended.txt | 54 +- .../q98.native_iceberg_compat/extended.txt | 54 +- .../q99.native_datafusion/extended.txt | 2 +- .../q1.native_datafusion/extended.txt | 111 +- .../q1.native_iceberg_compat/extended.txt | 106 +- .../q10.native_datafusion/extended.txt | 72 +- .../q10.native_iceberg_compat/extended.txt | 62 +- .../q11.native_datafusion/extended.txt | 203 +- .../q11.native_iceberg_compat/extended.txt | 195 +- .../q12.native_datafusion/extended.txt | 54 +- .../q12.native_iceberg_compat/extended.txt | 54 +- .../q13.native_datafusion/extended.txt | 88 +- .../q13.native_iceberg_compat/extended.txt | 88 +- .../q14a.native_datafusion/extended.txt | 1099 ++-- .../q14a.native_iceberg_compat/extended.txt | 1014 +-- .../q14b.native_datafusion/extended.txt | 796 ++- .../q14b.native_iceberg_compat/extended.txt | 741 ++- .../q15.native_datafusion/extended.txt | 66 +- .../q15.native_iceberg_compat/extended.txt | 66 +- .../q17.native_datafusion/extended.txt | 135 +- .../q17.native_iceberg_compat/extended.txt | 130 +- .../q18.native_datafusion/extended.txt | 107 +- .../q18.native_iceberg_compat/extended.txt | 107 +- .../q19.native_datafusion/extended.txt | 2 +- .../q20.native_datafusion/extended.txt | 54 +- .../q20.native_iceberg_compat/extended.txt | 54 +- .../q21.native_datafusion/extended.txt | 64 +- .../q21.native_iceberg_compat/extended.txt | 64 +- .../q22.native_datafusion/extended.txt | 68 +- .../q22.native_iceberg_compat/extended.txt | 68 +- .../q23a.native_datafusion/extended.txt | 203 +- .../q23a.native_iceberg_compat/extended.txt | 198 +- .../q23b.native_datafusion/extended.txt | 207 +- .../q23b.native_iceberg_compat/extended.txt | 202 +- .../q25.native_datafusion/extended.txt | 135 +- .../q25.native_iceberg_compat/extended.txt | 130 +- .../q26.native_datafusion/extended.txt | 81 +- .../q26.native_iceberg_compat/extended.txt | 81 +- .../q27.native_datafusion/extended.txt | 83 +- .../q27.native_iceberg_compat/extended.txt | 83 +- .../q29.native_datafusion/extended.txt | 140 +- .../q29.native_iceberg_compat/extended.txt | 140 +- .../q30.native_datafusion/extended.txt | 137 +- .../q30.native_iceberg_compat/extended.txt | 132 +- .../q31.native_datafusion/extended.txt | 282 +- .../q31.native_iceberg_compat/extended.txt | 270 +- .../q32.native_datafusion/extended.txt | 93 +- .../q32.native_iceberg_compat/extended.txt | 88 +- .../q33.native_datafusion/extended.txt | 217 +- .../q33.native_iceberg_compat/extended.txt | 207 +- .../q34.native_datafusion/extended.txt | 82 +- .../q34.native_iceberg_compat/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35.native_iceberg_compat/extended.txt | 62 +- .../q36.native_datafusion/extended.txt | 69 +- .../q36.native_iceberg_compat/extended.txt | 69 +- .../q37.native_datafusion/extended.txt | 64 +- .../q37.native_iceberg_compat/extended.txt | 64 +- .../q38.native_datafusion/extended.txt | 133 +- .../q38.native_iceberg_compat/extended.txt | 123 +- .../q39a.native_datafusion/extended.txt | 134 +- .../q39a.native_iceberg_compat/extended.txt | 134 +- .../q39b.native_datafusion/extended.txt | 134 +- .../q39b.native_iceberg_compat/extended.txt | 134 +- .../q4.native_datafusion/extended.txt | 299 +- .../q4.native_iceberg_compat/extended.txt | 283 +- .../q40.native_datafusion/extended.txt | 19 +- .../q40.native_iceberg_compat/extended.txt | 19 +- .../q45.native_datafusion/extended.txt | 66 +- .../q45.native_iceberg_compat/extended.txt | 66 +- .../q46.native_datafusion/extended.txt | 99 +- .../q46.native_iceberg_compat/extended.txt | 99 +- .../q47.native_datafusion/extended.txt | 179 +- .../q47.native_iceberg_compat/extended.txt | 179 +- .../q48.native_datafusion/extended.txt | 77 +- .../q48.native_iceberg_compat/extended.txt | 77 +- .../q49.native_datafusion/extended.txt | 154 +- .../q49.native_iceberg_compat/extended.txt | 144 +- .../q5.native_datafusion/extended.txt | 219 +- .../q5.native_iceberg_compat/extended.txt | 194 +- .../q50.native_datafusion/extended.txt | 77 +- .../q50.native_iceberg_compat/extended.txt | 77 +- .../q51.native_datafusion/extended.txt | 75 +- .../q51.native_iceberg_compat/extended.txt | 70 +- .../q53.native_datafusion/extended.txt | 65 +- .../q53.native_iceberg_compat/extended.txt | 65 +- .../q54.native_datafusion/extended.txt | 210 +- .../q54.native_iceberg_compat/extended.txt | 205 +- .../q56.native_datafusion/extended.txt | 223 +- .../q56.native_iceberg_compat/extended.txt | 213 +- .../q57.native_datafusion/extended.txt | 179 +- .../q57.native_iceberg_compat/extended.txt | 179 +- .../q58.native_datafusion/extended.txt | 258 +- .../q58.native_iceberg_compat/extended.txt | 239 +- .../q6.native_datafusion/extended.txt | 123 +- .../q6.native_iceberg_compat/extended.txt | 123 +- .../q60.native_datafusion/extended.txt | 223 +- .../q60.native_iceberg_compat/extended.txt | 213 +- .../q61.native_datafusion/extended.txt | 188 +- .../q61.native_iceberg_compat/extended.txt | 183 +- .../q62.native_datafusion/extended.txt | 2 +- .../q63.native_datafusion/extended.txt | 65 +- .../q63.native_iceberg_compat/extended.txt | 65 +- .../q64.native_datafusion/extended.txt | 70 +- .../q64.native_iceberg_compat/extended.txt | 62 +- .../q65.native_datafusion/extended.txt | 115 +- .../q65.native_iceberg_compat/extended.txt | 110 +- .../q66.native_datafusion/extended.txt | 154 +- .../q66.native_iceberg_compat/extended.txt | 150 +- .../q67.native_datafusion/extended.txt | 69 +- .../q67.native_iceberg_compat/extended.txt | 69 +- .../q68.native_datafusion/extended.txt | 99 +- .../q68.native_iceberg_compat/extended.txt | 99 +- .../q69.native_datafusion/extended.txt | 72 +- .../q69.native_iceberg_compat/extended.txt | 62 +- .../q7.native_datafusion/extended.txt | 81 +- .../q7.native_iceberg_compat/extended.txt | 81 +- .../q70.native_datafusion/extended.txt | 77 +- .../q70.native_iceberg_compat/extended.txt | 72 +- .../q71.native_datafusion/extended.txt | 119 +- .../q71.native_iceberg_compat/extended.txt | 109 +- .../q72.native_datafusion/extended.txt | 126 +- .../q72.native_iceberg_compat/extended.txt | 126 +- .../q73.native_datafusion/extended.txt | 82 +- .../q73.native_iceberg_compat/extended.txt | 82 +- .../q74.native_datafusion/extended.txt | 201 +- .../q74.native_iceberg_compat/extended.txt | 193 +- .../q75.native_datafusion/extended.txt | 240 +- .../q75.native_iceberg_compat/extended.txt | 216 +- .../q77.native_datafusion/extended.txt | 257 +- .../q77.native_iceberg_compat/extended.txt | 232 +- .../q78.native_datafusion/extended.txt | 49 +- .../q78.native_iceberg_compat/extended.txt | 37 +- .../q79.native_datafusion/extended.txt | 75 +- .../q79.native_iceberg_compat/extended.txt | 75 +- .../q8.native_datafusion/extended.txt | 106 +- .../q8.native_iceberg_compat/extended.txt | 106 +- .../q80.native_datafusion/extended.txt | 57 +- .../q80.native_iceberg_compat/extended.txt | 39 +- .../q81.native_datafusion/extended.txt | 137 +- .../q81.native_iceberg_compat/extended.txt | 132 +- .../q82.native_datafusion/extended.txt | 64 +- .../q82.native_iceberg_compat/extended.txt | 64 +- .../q83.native_datafusion/extended.txt | 261 +- .../q83.native_iceberg_compat/extended.txt | 247 +- .../q85.native_datafusion/extended.txt | 118 +- .../q85.native_iceberg_compat/extended.txt | 118 +- .../q86.native_datafusion/extended.txt | 56 +- .../q86.native_iceberg_compat/extended.txt | 56 +- .../q87.native_datafusion/extended.txt | 133 +- .../q87.native_iceberg_compat/extended.txt | 123 +- .../q89.native_datafusion/extended.txt | 65 +- .../q89.native_iceberg_compat/extended.txt | 65 +- .../q91.native_datafusion/extended.txt | 104 +- .../q91.native_iceberg_compat/extended.txt | 104 +- .../q92.native_datafusion/extended.txt | 93 +- .../q92.native_iceberg_compat/extended.txt | 88 +- .../q97.native_datafusion/extended.txt | 61 +- .../q97.native_iceberg_compat/extended.txt | 56 +- .../q98.native_datafusion/extended.txt | 54 +- .../q98.native_iceberg_compat/extended.txt | 54 +- .../q99.native_datafusion/extended.txt | 2 +- .../q10a.native_datafusion/extended.txt | 129 +- .../q10a.native_iceberg_compat/extended.txt | 119 +- .../q11.native_datafusion/extended.txt | 201 +- .../q11.native_iceberg_compat/extended.txt | 193 +- .../q12.native_datafusion/extended.txt | 54 +- .../q12.native_iceberg_compat/extended.txt | 54 +- .../q14.native_datafusion/extended.txt | 796 ++- .../q14.native_iceberg_compat/extended.txt | 741 ++- .../q14a.native_datafusion/extended.txt | 5515 ++++++++--------- .../q14a.native_iceberg_compat/extended.txt | 5115 ++++++++------- .../q18a.native_datafusion/extended.txt | 485 +- .../q18a.native_iceberg_compat/extended.txt | 465 +- .../q20.native_datafusion/extended.txt | 54 +- .../q20.native_iceberg_compat/extended.txt | 54 +- .../q22.native_datafusion/extended.txt | 42 +- .../q22.native_iceberg_compat/extended.txt | 42 +- .../q22a.native_datafusion/extended.txt | 344 +- .../q22a.native_iceberg_compat/extended.txt | 344 +- .../q27a.native_datafusion/extended.txt | 223 +- .../q27a.native_iceberg_compat/extended.txt | 213 +- .../q34.native_datafusion/extended.txt | 82 +- .../q34.native_iceberg_compat/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35.native_iceberg_compat/extended.txt | 62 +- .../q35a.native_datafusion/extended.txt | 129 +- .../q35a.native_iceberg_compat/extended.txt | 119 +- .../q36a.native_datafusion/extended.txt | 211 +- .../q36a.native_iceberg_compat/extended.txt | 211 +- .../q47.native_datafusion/extended.txt | 179 +- .../q47.native_iceberg_compat/extended.txt | 179 +- .../q49.native_datafusion/extended.txt | 154 +- .../q49.native_iceberg_compat/extended.txt | 144 +- .../q51a.native_datafusion/extended.txt | 294 +- .../q51a.native_iceberg_compat/extended.txt | 274 +- .../q57.native_datafusion/extended.txt | 179 +- .../q57.native_iceberg_compat/extended.txt | 179 +- .../q5a.native_datafusion/extended.txt | 654 +- .../q5a.native_iceberg_compat/extended.txt | 577 +- .../q6.native_datafusion/extended.txt | 123 +- .../q6.native_iceberg_compat/extended.txt | 123 +- .../q64.native_datafusion/extended.txt | 70 +- .../q64.native_iceberg_compat/extended.txt | 62 +- .../q67a.native_datafusion/extended.txt | 632 +- .../q67a.native_iceberg_compat/extended.txt | 632 +- .../q70a.native_datafusion/extended.txt | 227 +- .../q70a.native_iceberg_compat/extended.txt | 212 +- .../q72.native_datafusion/extended.txt | 126 +- .../q72.native_iceberg_compat/extended.txt | 126 +- .../q74.native_datafusion/extended.txt | 201 +- .../q74.native_iceberg_compat/extended.txt | 193 +- .../q75.native_datafusion/extended.txt | 240 +- .../q75.native_iceberg_compat/extended.txt | 216 +- .../q77a.native_datafusion/extended.txt | 767 +-- .../q77a.native_iceberg_compat/extended.txt | 692 ++- .../q78.native_datafusion/extended.txt | 49 +- .../q78.native_iceberg_compat/extended.txt | 37 +- .../q80a.native_datafusion/extended.txt | 167 +- .../q80a.native_iceberg_compat/extended.txt | 113 +- .../q86a.native_datafusion/extended.txt | 172 +- .../q86a.native_iceberg_compat/extended.txt | 172 +- .../q98.native_datafusion/extended.txt | 54 +- .../q98.native_iceberg_compat/extended.txt | 54 +- .../q10a.native_datafusion/extended.txt | 129 +- .../q10a.native_iceberg_compat/extended.txt | 119 +- .../q11.native_datafusion/extended.txt | 201 +- .../q11.native_iceberg_compat/extended.txt | 193 +- .../q12.native_datafusion/extended.txt | 54 +- .../q12.native_iceberg_compat/extended.txt | 54 +- .../q14.native_datafusion/extended.txt | 820 +-- .../q14.native_iceberg_compat/extended.txt | 749 ++- .../q14a.native_datafusion/extended.txt | 5515 ++++++++--------- .../q14a.native_iceberg_compat/extended.txt | 5115 ++++++++------- .../q18a.native_datafusion/extended.txt | 485 +- .../q18a.native_iceberg_compat/extended.txt | 465 +- .../q20.native_datafusion/extended.txt | 54 +- .../q20.native_iceberg_compat/extended.txt | 54 +- .../q22.native_datafusion/extended.txt | 42 +- .../q22.native_iceberg_compat/extended.txt | 42 +- .../q22a.native_datafusion/extended.txt | 344 +- .../q22a.native_iceberg_compat/extended.txt | 344 +- .../q27a.native_datafusion/extended.txt | 223 +- .../q27a.native_iceberg_compat/extended.txt | 213 +- .../q34.native_datafusion/extended.txt | 82 +- .../q34.native_iceberg_compat/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35.native_iceberg_compat/extended.txt | 62 +- .../q35a.native_datafusion/extended.txt | 129 +- .../q35a.native_iceberg_compat/extended.txt | 119 +- .../q36a.native_datafusion/extended.txt | 211 +- .../q36a.native_iceberg_compat/extended.txt | 211 +- .../q47.native_datafusion/extended.txt | 179 +- .../q47.native_iceberg_compat/extended.txt | 179 +- .../q49.native_datafusion/extended.txt | 154 +- .../q49.native_iceberg_compat/extended.txt | 144 +- .../q51a.native_datafusion/extended.txt | 294 +- .../q51a.native_iceberg_compat/extended.txt | 274 +- .../q57.native_datafusion/extended.txt | 179 +- .../q57.native_iceberg_compat/extended.txt | 179 +- .../q5a.native_datafusion/extended.txt | 654 +- .../q5a.native_iceberg_compat/extended.txt | 577 +- .../q6.native_datafusion/extended.txt | 141 +- .../q6.native_iceberg_compat/extended.txt | 127 +- .../q64.native_datafusion/extended.txt | 70 +- .../q64.native_iceberg_compat/extended.txt | 62 +- .../q67a.native_datafusion/extended.txt | 632 +- .../q67a.native_iceberg_compat/extended.txt | 632 +- .../q70a.native_datafusion/extended.txt | 227 +- .../q70a.native_iceberg_compat/extended.txt | 212 +- .../q72.native_datafusion/extended.txt | 126 +- .../q72.native_iceberg_compat/extended.txt | 126 +- .../q74.native_datafusion/extended.txt | 201 +- .../q74.native_iceberg_compat/extended.txt | 193 +- .../q75.native_datafusion/extended.txt | 240 +- .../q75.native_iceberg_compat/extended.txt | 216 +- .../q77a.native_datafusion/extended.txt | 767 +-- .../q77a.native_iceberg_compat/extended.txt | 692 ++- .../q78.native_datafusion/extended.txt | 49 +- .../q78.native_iceberg_compat/extended.txt | 37 +- .../q80a.native_datafusion/extended.txt | 167 +- .../q80a.native_iceberg_compat/extended.txt | 113 +- .../q86a.native_datafusion/extended.txt | 172 +- .../q86a.native_iceberg_compat/extended.txt | 172 +- .../q98.native_datafusion/extended.txt | 54 +- .../q98.native_iceberg_compat/extended.txt | 54 +- .../q10a.native_datafusion/extended.txt | 129 +- .../q10a.native_iceberg_compat/extended.txt | 119 +- .../q11.native_datafusion/extended.txt | 201 +- .../q11.native_iceberg_compat/extended.txt | 193 +- .../q12.native_datafusion/extended.txt | 54 +- .../q12.native_iceberg_compat/extended.txt | 54 +- .../q14.native_datafusion/extended.txt | 796 ++- .../q14.native_iceberg_compat/extended.txt | 741 ++- .../q14a.native_datafusion/extended.txt | 5515 ++++++++--------- .../q14a.native_iceberg_compat/extended.txt | 5115 ++++++++------- .../q18a.native_datafusion/extended.txt | 485 +- .../q18a.native_iceberg_compat/extended.txt | 465 +- .../q20.native_datafusion/extended.txt | 54 +- .../q20.native_iceberg_compat/extended.txt | 54 +- .../q22.native_datafusion/extended.txt | 42 +- .../q22.native_iceberg_compat/extended.txt | 42 +- .../q22a.native_datafusion/extended.txt | 344 +- .../q22a.native_iceberg_compat/extended.txt | 344 +- .../q27a.native_datafusion/extended.txt | 223 +- .../q27a.native_iceberg_compat/extended.txt | 213 +- .../q34.native_datafusion/extended.txt | 82 +- .../q34.native_iceberg_compat/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35.native_iceberg_compat/extended.txt | 62 +- .../q35a.native_datafusion/extended.txt | 129 +- .../q35a.native_iceberg_compat/extended.txt | 119 +- .../q36a.native_datafusion/extended.txt | 211 +- .../q36a.native_iceberg_compat/extended.txt | 211 +- .../q47.native_datafusion/extended.txt | 179 +- .../q47.native_iceberg_compat/extended.txt | 179 +- .../q49.native_datafusion/extended.txt | 154 +- .../q49.native_iceberg_compat/extended.txt | 144 +- .../q51a.native_datafusion/extended.txt | 294 +- .../q51a.native_iceberg_compat/extended.txt | 274 +- .../q57.native_datafusion/extended.txt | 179 +- .../q57.native_iceberg_compat/extended.txt | 179 +- .../q5a.native_datafusion/extended.txt | 654 +- .../q5a.native_iceberg_compat/extended.txt | 577 +- .../q6.native_datafusion/extended.txt | 123 +- .../q6.native_iceberg_compat/extended.txt | 123 +- .../q64.native_datafusion/extended.txt | 70 +- .../q64.native_iceberg_compat/extended.txt | 62 +- .../q67a.native_datafusion/extended.txt | 629 +- .../q67a.native_iceberg_compat/extended.txt | 629 +- .../q70a.native_datafusion/extended.txt | 227 +- .../q70a.native_iceberg_compat/extended.txt | 212 +- .../q72.native_datafusion/extended.txt | 126 +- .../q72.native_iceberg_compat/extended.txt | 126 +- .../q74.native_datafusion/extended.txt | 201 +- .../q74.native_iceberg_compat/extended.txt | 193 +- .../q75.native_datafusion/extended.txt | 240 +- .../q75.native_iceberg_compat/extended.txt | 216 +- .../q77a.native_datafusion/extended.txt | 767 +-- .../q77a.native_iceberg_compat/extended.txt | 692 ++- .../q78.native_datafusion/extended.txt | 49 +- .../q78.native_iceberg_compat/extended.txt | 37 +- .../q80a.native_datafusion/extended.txt | 167 +- .../q80a.native_iceberg_compat/extended.txt | 113 +- .../q86a.native_datafusion/extended.txt | 172 +- .../q86a.native_iceberg_compat/extended.txt | 172 +- .../q98.native_datafusion/extended.txt | 54 +- .../q98.native_iceberg_compat/extended.txt | 54 +- .../ParquetReadFromFakeHadoopFsSuite.scala | 6 +- 665 files changed, 71430 insertions(+), 67502 deletions(-) diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index d6cef721eb..d6ddca0d2b 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -62,7 +62,7 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { } val maxSplitBytes = FilePartition.maxSplitBytes(relation.sparkSession, filteredListing) - val splitFiles = filteredListing.filePartitionIterator + val splitFilesList = filteredListing.filePartitionIterator .flatMap { partition => partition.files.flatMap { file => val filePath = file.getPath @@ -82,7 +82,7 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { .toSeq .sortBy(_.length)(implicitly[Ordering[Long]].reverse) - FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) + FilePartition.getFilePartitions(relation.sparkSession, splitFilesList, maxSplitBytes) } private def isDynamicPruningFilter(e: Expression): Boolean = diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt index 0622aad9d5..901ca3692b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt @@ -1,61 +1,58 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt index 8aa14c43dd..18e33d8acc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt @@ -1,53 +1,61 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt index 0f9f19de77..55ce155c89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt @@ -1,106 +1,99 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt index 781eae9054..1755c11325 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt @@ -1,91 +1,106 @@ -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 - +- 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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt index 6c2a775097..087450c911 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..9c704a0ed6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt @@ -3,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt index 4c0d0b7a33..227d98e43b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt @@ -1,48 +1,42 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt index 08e9beb692..2ac8a365e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt @@ -1,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt index dfd3434d90..e16c7013b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt @@ -1,553 +1,554 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- Filter +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometProject + : +- CometFilter : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt index 4af04a7846..9525bd6927 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -1,469 +1,553 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- Filter : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt index c4138c254a..4b85046c30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt @@ -1,404 +1,394 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt index f56d229b68..2b714db2ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt @@ -1,339 +1,404 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- BroadcastHashJoin + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt index 8ed98bc5cb..47df1d6e25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt @@ -1,36 +1,32 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt index 6de0c64850..d87e8023b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt @@ -1,32 +1,36 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt index 26e924fd7e..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt index 4e1a4f3ec0..a45665ebb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt @@ -1,62 +1,70 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt index b47fce49b3..2a0b449e0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt @@ -1,58 +1,51 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt index 0b554c7e7c..907dceaab6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt @@ -1,51 +1,58 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt index fbe6798e3d..d76549df2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt @@ -33,6 +33,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.customer_address 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-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt index 369ec68bb4..79803b0fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt index cd52b2cd12..e768681167 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt @@ -3,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt index 51fcfd010a..4b4521d08b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt @@ -1,35 +1,31 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt index ef02334507..fddcd19496 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt @@ -1,31 +1,35 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt index 9b15a52cff..e03a649133 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt @@ -1,37 +1,33 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt index 7369619d2f..7901019f25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt @@ -1,33 +1,37 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt index aadeb13f7b..6151d372dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt @@ -8,80 +8,71 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -102,41 +93,41 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +148,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt index a84226b077..840aa90904 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt @@ -8,71 +8,80 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- 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.item + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- 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 - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -93,36 +102,41 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -143,4 +157,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt index d41bf0802a..cda2b25dfe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt @@ -10,81 +10,72 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- Filter - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -131,42 +122,42 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +200,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt index cd91f5de49..1cf3a75f81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt @@ -10,72 +10,81 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometExchange - : : : : +- 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 - : : : : +- 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.item + : : : : +- CometColumnarExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- Filter + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- 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 - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -122,37 +131,42 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -195,4 +209,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt index 26e924fd7e..6b601f21a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt index 4e1a4f3ec0..a45665ebb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt @@ -1,62 +1,70 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt index 3b9d40795e..bc4d4fdfb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt index dbea7afe34..88c83f0c72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt @@ -1,39 +1,44 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt index a4946e4771..4aba7f0f85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt @@ -1,45 +1,40 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt index f550f3855c..3c0aa60951 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt @@ -1,40 +1,45 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt index dacaab515f..6b601f21a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt @@ -1,75 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt index fe03fef8a7..6d91a6198e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt @@ -1,67 +1,75 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt index 8373409822..49560ff5d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt index 7ebd69d340..164bb96ead 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt index f6f3c8032a..3fc7ae5a54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt @@ -1,150 +1,138 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- HashAggregate - : : : : : +- CometColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt index b4e7aa4db6..ee16bf1f4d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt @@ -1,126 +1,150 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- 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 + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- HashAggregate + : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : : +- BroadcastExchange + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt index 1eb8f397c3..3ef52bc501 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt index 943bb147ff..8f187b8e66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt @@ -1,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt index 79b2700a8c..57033e3622 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt @@ -1,112 +1,107 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt index 063fdb6faf..423f4c671b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt @@ -1,97 +1,112 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- 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 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt index bd826eb3a3..518e9ce811 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt @@ -3,41 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt index 89c523a388..acaca69a96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt @@ -3,36 +3,41 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt index fb7dbb8d72..dd5c2f4349 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt index abe0937d1d..64e799aa8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt @@ -1,34 +1,38 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt index 061475900e..d976fd7f57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt @@ -6,74 +6,75 @@ CometColumnarToRow +- CometBroadcastHashJoin :- CometBroadcastHashJoin : :- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt index 55af64efc6..bf503d1bfa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt @@ -6,65 +6,74 @@ CometColumnarToRow +- CometBroadcastHashJoin :- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt index e44cca98a1..f28434c7c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt index 8ff1442cf8..80c6af0c0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- BroadcastHashJoin + :- Project + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt index e44cca98a1..f28434c7c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt index 8ff1442cf8..80c6af0c0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- BroadcastHashJoin + :- Project + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt index 54e1a1c11a..6acf46921e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt @@ -1,154 +1,147 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- HashAggregate - : : : : : +- CometColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt index 1be0816ee1..54522ad631 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt @@ -1,131 +1,154 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- 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.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- HashAggregate + : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt index d5dbd76871..598f4b63a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt @@ -12,15 +12,14 @@ CometColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -38,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt index 54a7b21486..9c1c81dcdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt @@ -12,14 +12,15 @@ CometColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometExchange - : : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -37,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt index c58a64e326..4b7559eccd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt @@ -5,40 +5,36 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -48,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt index 4a57091056..241c4e86f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt @@ -5,36 +5,40 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -44,4 +48,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt index c122bf3803..d35968a3da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan 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 + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt index c842000e11..adadb2dc4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt @@ -1,49 +1,56 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt index 092083e465..33a209d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt index b50b570b4b..a3eb870b39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt @@ -10,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt index 1ef8426231..f39a6f8a31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt index 57fbc2d035..c940db4cf1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt @@ -1,37 +1,42 @@ -CometColumnarToRow -+- 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 - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- 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_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt index 6c4ded1515..ff7ad3b219 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt index 75684a9669..b10cf058e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt @@ -11,31 +11,35 @@ 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -43,26 +47,30 @@ CometColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -70,25 +78,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt index 60352ef90a..45e5c0f0e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt @@ -1,106 +1,115 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt index d128acaf06..ffeff7c4fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt @@ -1,90 +1,106 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Union + : : :- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt index 6ece21441c..07d160c357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt index 0fbba7d1e8..c90a5e8c49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt @@ -1,37 +1,42 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- 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.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt index 9dc007f5e9..40b425a184 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt @@ -12,48 +12,47 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt index 66c5717cc7..2936547d40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt @@ -12,42 +12,48 @@ 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt index 825b1ed81c..e2f1062058 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt @@ -4,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt index 6ba04a3153..f5e651b035 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt @@ -1,116 +1,96 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Union - : : : : : : : :- Project - : : : : : : : : +- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : +- Project - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : :- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +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 + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : :- ReusedSubquery + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 51 out of 96 eligible operators (53%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 88 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt index 98fce92fb1..0bf0e75168 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt @@ -1,91 +1,116 @@ -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 - : : : : :- 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 - : : : +- 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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Union + : : : : : : : :- Project + : : : : : : : : +- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- Project + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : :- 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 + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : :- 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 + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 96 eligible operators (53%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt index 7a2832757d..59e112c499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt index 36b9a8306d..1ce83c5a0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt @@ -1,100 +1,115 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- 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 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt index 65c66a7da8..a6f94c745d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt index de7d26cd06..5234ab1b35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt @@ -10,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt index 4b333b4991..72bd7e9cba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt @@ -1,123 +1,142 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 104 eligible operators (53%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 118 out of 130 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt index 3d3dd68990..e310b0bf2a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt @@ -1,112 +1,123 @@ -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 - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- 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 - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- 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 - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 96 out of 104 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 104 eligible operators (53%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt index fe3b28849a..6c0981f51f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt @@ -1,68 +1,57 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..21ba0466c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt @@ -1,57 +1,68 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt index 7a2832757d..59e112c499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt index 36b9a8306d..1ce83c5a0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt @@ -1,100 +1,115 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- 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 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt index 0443f5fc55..2d4c85b5a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt @@ -1,101 +1,93 @@ Project -+- BroadcastNestedLoopJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt index 7bae7d5724..a29873fa6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt @@ -1,88 +1,101 @@ Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- 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.item +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- 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.item -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt index 52d7828b06..0bfc07426a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site + : +- CometNativeScan parquet spark_catalog.default.warehouse +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt index 825b1ed81c..e2f1062058 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt @@ -4,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt index 6e3f37148b..8aa248f6ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -117,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.income_band + : : : +- CometNativeScan parquet spark_catalog.default.promotion : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -162,22 +160,20 @@ CometColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -239,13 +235,13 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt index 059acab385..da58064931 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt @@ -40,20 +40,22 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -160,20 +162,22 @@ CometColumnarToRow : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -244,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt index b1bae81b20..402cc842ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt @@ -1,60 +1,57 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt index 1fc09bd2e3..9a5a06cf91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt @@ -1,52 +1,60 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt index caf49b6d52..3d8af2a097 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt @@ -1,82 +1,74 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.ship_mode - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.ship_mode +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.ship_mode + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt index ca39e001b0..d77f6eab96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt @@ -1,70 +1,82 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : +- 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.time_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : +- 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.time_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt index e64576b0f0..5643bbe8e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt @@ -5,42 +5,38 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 15 out of 37 eligible operators (40%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt index 7504243c7a..2a0632a3bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt @@ -5,38 +5,42 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- 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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 37 eligible operators (40%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt index c122bf3803..d35968a3da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan 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 + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt index c842000e11..adadb2dc4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt @@ -1,49 +1,56 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt index 821b101f54..67d364800e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt @@ -9,46 +9,52 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,4 +69,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt index b4fbb455ef..63b1420137 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt @@ -9,42 +9,46 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -59,4 +63,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt index f40fdbe4e3..35091ce142 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt index 655f651a41..59da61ba7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt @@ -1,39 +1,44 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt index 4832534e6e..f0b51ba168 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt @@ -11,19 +11,18 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,29 +36,31 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 53 eligible operators (33%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 57 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt index 45a2c7a669..84bf0cc584 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt @@ -11,18 +11,19 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -36,26 +37,29 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 53 eligible operators (33%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt index 427d3518dc..5d11aad358 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt @@ -1,62 +1,63 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt index 25ea15c536..673dbd4ada 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt @@ -1,53 +1,62 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt index 081972705a..7a49cad749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt index 9000d27d1b..c2058aa3a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt @@ -6,67 +6,77 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Filter + : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt index 476c7be954..12b5755eb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt @@ -1,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt index 9f6954be38..33a79ee1f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,55 +42,57 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometSort +- CometExchange +- CometFilter @@ -107,28 +106,25 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,54 +133,56 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt index 22ccc17641..7afcaebec8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt @@ -15,25 +15,28 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -42,21 +45,24 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -65,21 +71,24 @@ CometColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -98,25 +107,28 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -125,21 +137,24 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -148,25 +163,28 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt index d12b8dde24..4b08389eae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt @@ -5,131 +5,138 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt index ed8a9e38ca..5ba92ec55f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt @@ -5,113 +5,131 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange + : +- BroadcastHashJoin + : :- HashAggregate : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin + : :- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt index 65256f2124..c253b8aff8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,16 +41,19 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -66,18 +68,21 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..eec4fb502f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt @@ -14,14 +14,15 @@ TakeOrderedAndProject : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -41,10 +42,11 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery + : : : +- CometColumnarExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -64,10 +66,11 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery + : : +- CometColumnarExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -77,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt index 05bd194c34..add26d22d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt index 012403275a..235c1e92f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt @@ -1,39 +1,44 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt index 257b7f0d77..86b85906aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt @@ -1,56 +1,52 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt index 8e04bbeba7..a240608c19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt @@ -1,52 +1,56 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- 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 - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt index 9d97e27318..5ffd41f4ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt @@ -19,16 +19,15 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -64,16 +63,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -81,7 +84,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,16 +107,20 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -121,7 +128,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_site + : : +- CometNativeScan parquet spark_catalog.default.store : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -131,4 +138,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt index ae6fc19b60..fccbf0a389 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt @@ -19,15 +19,16 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -63,10 +64,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -102,10 +104,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery + : : : : : +- CometColumnarExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -128,4 +131,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt index 0e70ec42fa..aab91b60e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt index 9708802e79..e6286a9886 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt index c86e7ba2cb..3fa31193d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt index 9fbb7893b8..4da4d56042 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt @@ -1,34 +1,38 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt index 1cb87c7eb1..dc88a9234f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt @@ -1,116 +1,133 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 60 out of 101 eligible operators (59%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt index 924b12cf01..ff077f9dd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt @@ -1,105 +1,116 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 60 out of 101 eligible operators (59%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt index 7f8b90a86c..dde36ec726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt @@ -1,64 +1,56 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- BroadcastExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.reason +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_page + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt index e2383394d1..3797565aed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt @@ -1,56 +1,64 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- BroadcastExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : : +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt index 41081debd9..86dc323961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt @@ -3,34 +3,30 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt index dfc810b108..e899c49c4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt @@ -3,30 +3,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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt index 638b8865e0..05d07c8d36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt @@ -7,76 +7,77 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometColumnarToRow : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometColumnarToRow : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange +- CometColumnarToRow +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt index ddee139acf..e2b06cf20d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt @@ -7,67 +7,76 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometColumnarToRow : : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometColumnarToRow : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange +- CometColumnarToRow +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt index 825b1ed81c..e2f1062058 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt @@ -4,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt index 7499c5aafc..9f7d7f87c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt @@ -1,59 +1,51 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt index a1c6b73d37..c0b4c3085d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt @@ -1,51 +1,59 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt index 2d15266049..5695ae40ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt index 20df9a8047..874eb62d3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt @@ -1,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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 - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt index 798bfdaf1b..9fe719d3c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt @@ -6,36 +6,37 @@ CometColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.catalog_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt index e1aefb8382..82e4b4b23c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt @@ -6,32 +6,36 @@ CometColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- ColumnarToRow + : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt index 030031856f..fc4889c2d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt @@ -6,33 +6,29 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt index 4c972848e7..e4611ec552 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt @@ -6,29 +6,33 @@ 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt index c075163849..6059603410 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometNativeScan parquet spark_catalog.default.warehouse +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt index 0622aad9d5..901ca3692b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt @@ -1,61 +1,58 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 8aa14c43dd..18e33d8acc 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,53 +1,61 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt index 0f9f19de77..55ce155c89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt @@ -1,106 +1,99 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 781eae9054..1755c11325 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,91 +1,106 @@ -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 - +- 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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt index 6c2a775097..087450c911 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 3f41c97ff5..9c704a0ed6 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,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). 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/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt index 4c0d0b7a33..227d98e43b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt @@ -1,48 +1,42 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 08e9beb692..2ac8a365e3 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,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt index dfd3434d90..e16c7013b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt @@ -1,553 +1,554 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- Filter +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometProject + : +- CometFilter : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt index 4af04a7846..9525bd6927 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt @@ -1,469 +1,553 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- Filter : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt index e44d85bdc2..6d6eda80ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt @@ -1,408 +1,414 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 337 eligible operators (37%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 343 out of 387 eligible operators (88%). Final plan contains 25 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt index b32dd95954..a02b0fda3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt @@ -1,343 +1,408 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- ReusedSubquery +TakeOrderedAndProject ++- BroadcastHashJoin + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- 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 + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : +- 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 -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 337 eligible operators (37%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt index 8ed98bc5cb..47df1d6e25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt @@ -1,36 +1,32 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 6de0c64850..d87e8023b8 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,32 +1,36 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt index 26e924fd7e..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt index 4e1a4f3ec0..a45665ebb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt @@ -1,62 +1,70 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt index b47fce49b3..2a0b449e0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt @@ -1,58 +1,51 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt index 0b554c7e7c..907dceaab6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt @@ -1,51 +1,58 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt index fbe6798e3d..d76549df2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt @@ -33,6 +33,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.customer_address 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/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt index 369ec68bb4..79803b0fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 cd52b2cd12..e768681167 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,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). 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/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt index 51fcfd010a..4b4521d08b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt @@ -1,35 +1,31 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt index ef02334507..fddcd19496 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt @@ -1,31 +1,35 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt index 9b15a52cff..e03a649133 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt @@ -1,37 +1,33 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt index 7369619d2f..7901019f25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt @@ -1,33 +1,37 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt index aadeb13f7b..6151d372dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt @@ -8,80 +8,71 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -102,41 +93,41 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +148,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt index a84226b077..840aa90904 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt @@ -8,71 +8,80 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- 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.item + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- 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 - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -93,36 +102,41 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -143,4 +157,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt index d41bf0802a..cda2b25dfe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt @@ -10,81 +10,72 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- Filter - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -131,42 +122,42 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +200,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt index cd91f5de49..1cf3a75f81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt @@ -10,72 +10,81 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometExchange - : : : : +- 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 - : : : : +- 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.item + : : : : +- CometColumnarExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- Filter + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- 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 - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -122,37 +131,42 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -195,4 +209,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt index 26e924fd7e..6b601f21a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 4e1a4f3ec0..a45665ebb8 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,62 +1,70 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt index 3b9d40795e..bc4d4fdfb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt index dbea7afe34..88c83f0c72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt @@ -1,39 +1,44 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt index a4946e4771..4aba7f0f85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt @@ -1,45 +1,40 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt index f550f3855c..3c0aa60951 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt @@ -1,40 +1,45 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt index dacaab515f..6b601f21a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt @@ -1,75 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt index fe03fef8a7..6d91a6198e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt @@ -1,67 +1,75 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt index 8373409822..49560ff5d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 7ebd69d340..164bb96ead 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,65 +1,75 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt index f6f3c8032a..3fc7ae5a54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt @@ -1,150 +1,138 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- HashAggregate - : : : : : +- CometColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 b4e7aa4db6..ee16bf1f4d 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,126 +1,150 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- 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 + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- HashAggregate + : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : : +- BroadcastExchange + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt index 1eb8f397c3..3ef52bc501 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 943bb147ff..8f187b8e66 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,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt index 79b2700a8c..57033e3622 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt @@ -1,112 +1,107 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 063fdb6faf..423f4c671b 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,97 +1,112 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- 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 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt index bd826eb3a3..518e9ce811 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt @@ -3,41 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 89c523a388..acaca69a96 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,36 +3,41 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt index fb7dbb8d72..dd5c2f4349 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt index abe0937d1d..64e799aa8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt @@ -1,34 +1,38 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt index 061475900e..d976fd7f57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt @@ -6,74 +6,75 @@ CometColumnarToRow +- CometBroadcastHashJoin :- CometBroadcastHashJoin : :- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt index 55af64efc6..bf503d1bfa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt @@ -6,65 +6,74 @@ CometColumnarToRow +- CometBroadcastHashJoin :- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt index e44cca98a1..f28434c7c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt index 8ff1442cf8..80c6af0c0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- BroadcastHashJoin + :- Project + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt index e44cca98a1..f28434c7c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt index 8ff1442cf8..80c6af0c0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- BroadcastHashJoin + :- Project + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt index 54e1a1c11a..6acf46921e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt @@ -1,154 +1,147 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- HashAggregate - : : : : : +- CometColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt index 1be0816ee1..54522ad631 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt @@ -1,131 +1,154 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- 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.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- HashAggregate + : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt index d5dbd76871..598f4b63a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt @@ -12,15 +12,14 @@ CometColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -38,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt index 54a7b21486..9c1c81dcdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt @@ -12,14 +12,15 @@ CometColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometExchange - : : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -37,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt index c58a64e326..4b7559eccd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt @@ -5,40 +5,36 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -48,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt index 4a57091056..241c4e86f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt @@ -5,36 +5,40 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -44,4 +48,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt index c122bf3803..d35968a3da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan 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 + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 c842000e11..adadb2dc4b 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,49 +1,56 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt index 092083e465..33a209d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 b50b570b4b..a3eb870b39 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,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt index 1ef8426231..f39a6f8a31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt index 57fbc2d035..c940db4cf1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt @@ -1,37 +1,42 @@ -CometColumnarToRow -+- 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 - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- 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_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt index 6c4ded1515..ff7ad3b219 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt index 75684a9669..b10cf058e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt @@ -11,31 +11,35 @@ 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -43,26 +47,30 @@ CometColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -70,25 +78,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt index 60352ef90a..45e5c0f0e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt @@ -1,106 +1,115 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 d128acaf06..ffeff7c4fd 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,90 +1,106 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Union + : : :- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt index 6ece21441c..07d160c357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt index 0fbba7d1e8..c90a5e8c49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt @@ -1,37 +1,42 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- 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.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt index 9dc007f5e9..40b425a184 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt @@ -12,48 +12,47 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 66c5717cc7..2936547d40 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,42 +12,48 @@ 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 825b1ed81c..e2f1062058 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,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt index 51c5ebdbea..9f071b0261 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt @@ -1,120 +1,128 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Union - : : : : : : : :- Project - : : : : : : : : +- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : +- Project - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- ReusedSubquery - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - :- Subquery - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow +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 + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : :- ReusedSubquery + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim + :- Subquery + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometColumnarToRow + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 51 out of 100 eligible operators (51%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 102 out of 116 eligible operators (87%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 a49bbabd10..6ecaf6925b 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,95 +1,120 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Union + : : : : : : : :- Project + : : : : : : : : +- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- Project + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- 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 + : : +- 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.store + +- 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 -Comet accelerated 75 out of 88 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 100 eligible operators (51%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt index 7a2832757d..59e112c499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 36b9a8306d..1ce83c5a0e 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,100 +1,115 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- 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 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt index 65c66a7da8..a6f94c745d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 de7d26cd06..5234ab1b35 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,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt index 787f30c437..7f74b937ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt @@ -1,127 +1,160 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- ReusedSubquery + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 108 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 127 out of 145 eligible operators (87%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 b4255cb7a1..83febdbd98 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,116 +1,127 @@ -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 - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- 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 + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- 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 + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- 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 + +- BroadcastExchange + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 -Comet accelerated 96 out of 108 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 108 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt index ee120ac920..75418138a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt @@ -1,70 +1,73 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 60 eligible operators (65%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 60 out of 66 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt index 2d049f149f..2191f4ba49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt @@ -1,59 +1,70 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- 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 - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- 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 + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- 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 + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 60 eligible operators (65%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt index 7a2832757d..59e112c499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 36b9a8306d..1ce83c5a0e 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,100 +1,115 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- 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 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt index 0443f5fc55..2d4c85b5a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt @@ -1,101 +1,93 @@ Project -+- BroadcastNestedLoopJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 7bae7d5724..a29873fa6e 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,88 +1,101 @@ Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- 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.item +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- 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.item -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt index 52d7828b06..0bfc07426a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site + : +- CometNativeScan parquet spark_catalog.default.warehouse +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 825b1ed81c..e2f1062058 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,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt index 6e3f37148b..8aa248f6ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -117,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.income_band + : : : +- CometNativeScan parquet spark_catalog.default.promotion : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -162,22 +160,20 @@ CometColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -239,13 +235,13 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 059acab385..da58064931 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 @@ -40,20 +40,22 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -160,20 +162,22 @@ CometColumnarToRow : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -244,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt index b1bae81b20..402cc842ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt @@ -1,60 +1,57 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 1fc09bd2e3..9a5a06cf91 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,52 +1,60 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt index caf49b6d52..3d8af2a097 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt @@ -1,82 +1,74 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.ship_mode - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.ship_mode +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.ship_mode + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt index ca39e001b0..d77f6eab96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt @@ -1,70 +1,82 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : +- 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.time_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : +- 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.time_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt index e64576b0f0..5643bbe8e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt @@ -5,42 +5,38 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 15 out of 37 eligible operators (40%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt index 7504243c7a..2a0632a3bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt @@ -5,38 +5,42 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- 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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 37 eligible operators (40%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt index c122bf3803..d35968a3da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan 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 + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 c842000e11..adadb2dc4b 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,49 +1,56 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt index 821b101f54..67d364800e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt @@ -9,46 +9,52 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,4 +69,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt index b4fbb455ef..63b1420137 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt @@ -9,42 +9,46 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -59,4 +63,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt index f40fdbe4e3..35091ce142 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt index 655f651a41..59da61ba7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt @@ -1,39 +1,44 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt index 4832534e6e..f0b51ba168 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt @@ -11,19 +11,18 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,29 +36,31 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 53 eligible operators (33%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 57 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 45a2c7a669..84bf0cc584 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 @@ -11,18 +11,19 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -36,26 +37,29 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 53 eligible operators (33%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt index 427d3518dc..5d11aad358 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt @@ -1,62 +1,63 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 25ea15c536..673dbd4ada 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,53 +1,62 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt index 081972705a..7a49cad749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt index 9000d27d1b..c2058aa3a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt @@ -6,67 +6,77 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Filter + : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 476c7be954..12b5755eb7 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,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt index 9f6954be38..33a79ee1f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,55 +42,57 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometSort +- CometExchange +- CometFilter @@ -107,28 +106,25 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,54 +133,56 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 22ccc17641..7afcaebec8 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 @@ -15,25 +15,28 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -42,21 +45,24 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -65,21 +71,24 @@ CometColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -98,25 +107,28 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -125,21 +137,24 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -148,25 +163,28 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt index d12b8dde24..4b08389eae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt @@ -5,131 +5,138 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 ed8a9e38ca..5ba92ec55f 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,113 +5,131 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange + : +- BroadcastHashJoin + : :- HashAggregate : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin + : :- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt index 65256f2124..c253b8aff8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,16 +41,19 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -66,18 +68,21 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 9d5ba53be1..eec4fb502f 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 @@ -14,14 +14,15 @@ TakeOrderedAndProject : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -41,10 +42,11 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery + : : : +- CometColumnarExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -64,10 +66,11 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery + : : +- CometColumnarExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -77,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt index 05bd194c34..add26d22d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 012403275a..235c1e92f2 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,39 +1,44 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt index 257b7f0d77..86b85906aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt @@ -1,56 +1,52 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 8e04bbeba7..a240608c19 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,52 +1,56 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- 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 - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt index 9d97e27318..5ffd41f4ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt @@ -19,16 +19,15 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -64,16 +63,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -81,7 +84,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,16 +107,20 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -121,7 +128,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_site + : : +- CometNativeScan parquet spark_catalog.default.store : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -131,4 +138,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 ae6fc19b60..fccbf0a389 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 @@ -19,15 +19,16 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -63,10 +64,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -102,10 +104,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery + : : : : : +- CometColumnarExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -128,4 +131,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt index 0e70ec42fa..aab91b60e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 9708802e79..e6286a9886 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,65 +1,75 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt index c86e7ba2cb..3fa31193d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt index 9fbb7893b8..4da4d56042 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt @@ -1,34 +1,38 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 30 eligible operators (50%). 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/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt index 1cb87c7eb1..dc88a9234f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt @@ -1,116 +1,133 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 60 out of 101 eligible operators (59%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt index 924b12cf01..ff077f9dd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt @@ -1,105 +1,116 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 60 out of 101 eligible operators (59%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt index 7f8b90a86c..dde36ec726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt @@ -1,64 +1,56 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- BroadcastExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.reason +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_page + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt index e2383394d1..3797565aed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt @@ -1,56 +1,64 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- BroadcastExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : : +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt index 41081debd9..86dc323961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt @@ -3,34 +3,30 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 dfc810b108..e899c49c4e 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,30 +3,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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt index 638b8865e0..05d07c8d36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt @@ -7,76 +7,77 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometColumnarToRow : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometColumnarToRow : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange +- CometColumnarToRow +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt index ddee139acf..e2b06cf20d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt @@ -7,67 +7,76 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometColumnarToRow : : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometColumnarToRow : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange +- CometColumnarToRow +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 825b1ed81c..e2f1062058 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,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt index 7499c5aafc..9f7d7f87c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt @@ -1,59 +1,51 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 a1c6b73d37..c0b4c3085d 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,51 +1,59 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt index 2d15266049..5695ae40ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 20df9a8047..874eb62d3c 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,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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 - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt index 798bfdaf1b..9fe719d3c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt @@ -6,36 +6,37 @@ CometColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.catalog_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt index e1aefb8382..82e4b4b23c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt @@ -6,32 +6,36 @@ CometColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- ColumnarToRow + : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt index 030031856f..fc4889c2d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt @@ -6,33 +6,29 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 4c972848e7..e4611ec552 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,29 +6,33 @@ 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt index c075163849..6059603410 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometNativeScan parquet spark_catalog.default.warehouse +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt index 0622aad9d5..901ca3692b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt @@ -1,61 +1,58 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt index 8aa14c43dd..18e33d8acc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt @@ -1,53 +1,61 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt index 0f9f19de77..55ce155c89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt @@ -1,106 +1,99 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt index 781eae9054..1755c11325 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt @@ -1,91 +1,106 @@ -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 - +- 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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt index 6c2a775097..087450c911 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..9c704a0ed6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt @@ -3,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). 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/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt index 4c0d0b7a33..227d98e43b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt @@ -1,48 +1,42 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt index 08e9beb692..2ac8a365e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt @@ -1,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index dfd3434d90..e16c7013b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt @@ -1,553 +1,554 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- Filter +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometProject + : +- CometFilter : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt index 4af04a7846..9525bd6927 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt @@ -1,469 +1,553 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- Filter : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt index c4138c254a..4b85046c30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt @@ -1,404 +1,394 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt index f56d229b68..2b714db2ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt @@ -1,339 +1,404 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- BroadcastHashJoin + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt index 8ed98bc5cb..47df1d6e25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt @@ -1,36 +1,32 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt index 6de0c64850..d87e8023b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt @@ -1,32 +1,36 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt index 26e924fd7e..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt index 4e1a4f3ec0..a45665ebb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt @@ -1,62 +1,70 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt index b47fce49b3..2a0b449e0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt @@ -1,58 +1,51 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt index 0b554c7e7c..907dceaab6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt @@ -1,51 +1,58 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt index fbe6798e3d..d76549df2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt @@ -33,6 +33,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.customer_address 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/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt index 369ec68bb4..79803b0fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt index cd52b2cd12..e768681167 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt @@ -3,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). 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/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt index 51fcfd010a..4b4521d08b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt @@ -1,35 +1,31 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt index ef02334507..fddcd19496 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt @@ -1,31 +1,35 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt index 9b15a52cff..e03a649133 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt @@ -1,37 +1,33 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt index 7369619d2f..7901019f25 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt @@ -1,33 +1,37 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt index aadeb13f7b..6151d372dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt @@ -8,80 +8,71 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -102,41 +93,41 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +148,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt index a84226b077..840aa90904 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt @@ -8,71 +8,80 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- 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.item + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- 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 - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -93,36 +102,41 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -143,4 +157,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt index d41bf0802a..cda2b25dfe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt @@ -10,81 +10,72 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- Filter - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -131,42 +122,42 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +200,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt index cd91f5de49..1cf3a75f81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt @@ -10,72 +10,81 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometExchange - : : : : +- 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 - : : : : +- 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.item + : : : : +- CometColumnarExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- Filter + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- 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 - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -122,37 +131,42 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -195,4 +209,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index 26e924fd7e..6b601f21a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt index 4e1a4f3ec0..a45665ebb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt @@ -1,62 +1,70 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt index 3b9d40795e..bc4d4fdfb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt index dbea7afe34..88c83f0c72 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt @@ -1,39 +1,44 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt index a4946e4771..4aba7f0f85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt @@ -1,45 +1,40 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt index f550f3855c..3c0aa60951 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt @@ -1,40 +1,45 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt index dacaab515f..6b601f21a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt @@ -1,75 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt index fe03fef8a7..6d91a6198e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt @@ -1,67 +1,75 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt index 8373409822..49560ff5d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt index 7ebd69d340..164bb96ead 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index f6f3c8032a..3fc7ae5a54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt @@ -1,150 +1,138 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- HashAggregate - : : : : : +- CometColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt index b4e7aa4db6..ee16bf1f4d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt @@ -1,126 +1,150 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- 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 + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- HashAggregate + : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : : +- BroadcastExchange + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt index 1eb8f397c3..3ef52bc501 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt index 943bb147ff..8f187b8e66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt @@ -1,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt index 79b2700a8c..89aa813e43 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt @@ -1,112 +1,107 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt index 063fdb6faf..423f4c671b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt @@ -1,97 +1,112 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- 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 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt index bd826eb3a3..518e9ce811 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt @@ -3,41 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt index 89c523a388..acaca69a96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt @@ -3,36 +3,41 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt index fb7dbb8d72..dd5c2f4349 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt index abe0937d1d..64e799aa8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt @@ -1,34 +1,38 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt index 061475900e..d976fd7f57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt @@ -6,74 +6,75 @@ CometColumnarToRow +- CometBroadcastHashJoin :- CometBroadcastHashJoin : :- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt index 55af64efc6..bf503d1bfa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt @@ -6,65 +6,74 @@ CometColumnarToRow +- CometBroadcastHashJoin :- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt index e44cca98a1..f28434c7c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt index 8ff1442cf8..80c6af0c0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- BroadcastHashJoin + :- Project + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt index e44cca98a1..f28434c7c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt index 8ff1442cf8..80c6af0c0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- BroadcastHashJoin + :- Project + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index 54e1a1c11a..6acf46921e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt @@ -1,154 +1,147 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- HashAggregate - : : : : : +- CometColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt index 1be0816ee1..54522ad631 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt @@ -1,131 +1,154 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- 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.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- HashAggregate + : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : : +- BroadcastExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- HashAggregate + : : : : +- CometColumnarToRow + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt index d5dbd76871..598f4b63a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt @@ -12,15 +12,14 @@ CometColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -38,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt index 54a7b21486..9c1c81dcdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt @@ -12,14 +12,15 @@ CometColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometExchange - : : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -37,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt index c58a64e326..4b7559eccd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt @@ -5,40 +5,36 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -48,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt index 4a57091056..241c4e86f0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt @@ -5,36 +5,40 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -44,4 +48,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt index c122bf3803..d35968a3da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan 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 + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt index c842000e11..adadb2dc4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt @@ -1,49 +1,56 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt index 092083e465..33a209d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt index b50b570b4b..a3eb870b39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt @@ -10,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt index 1ef8426231..f39a6f8a31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt index 57fbc2d035..c940db4cf1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt @@ -1,37 +1,42 @@ -CometColumnarToRow -+- 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 - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- 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_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index 6c4ded1515..ff7ad3b219 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt index 75684a9669..b10cf058e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt @@ -11,31 +11,35 @@ 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -43,26 +47,30 @@ CometColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -70,25 +78,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt index 60352ef90a..45e5c0f0e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt @@ -1,106 +1,115 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt index d128acaf06..ffeff7c4fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt @@ -1,90 +1,106 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Union + : : :- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt index 6ece21441c..07d160c357 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt index 0fbba7d1e8..c90a5e8c49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt @@ -1,37 +1,42 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- 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.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index 9dc007f5e9..40b425a184 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -12,48 +12,47 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt index 66c5717cc7..2936547d40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt @@ -12,42 +12,48 @@ 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt index 825b1ed81c..e2f1062058 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt @@ -4,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt index 6ba04a3153..f5e651b035 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt @@ -1,116 +1,96 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Union - : : : : : : : :- Project - : : : : : : : : +- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : +- Project - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : :- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +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 + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : :- ReusedSubquery + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 51 out of 96 eligible operators (53%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 88 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt index 98fce92fb1..0bf0e75168 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt @@ -1,91 +1,116 @@ -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 - : : : : :- 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 - : : : +- 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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Union + : : : : : : : :- Project + : : : : : : : : +- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- Project + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : :- 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 + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : :- 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 + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 96 eligible operators (53%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt index 7a2832757d..59e112c499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt index 36b9a8306d..1ce83c5a0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt @@ -1,100 +1,115 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- 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 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt index 65c66a7da8..a6f94c745d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt index de7d26cd06..5234ab1b35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt @@ -10,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt index ed82c835cc..bfca848535 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt @@ -1,126 +1,134 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 58 out of 108 eligible operators (53%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 112 out of 124 eligible operators (90%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt index c2dd549ac6..fb001bbd5c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt @@ -1,115 +1,126 @@ -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 - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- 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 - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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 - : : +- 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 - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- 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 + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- 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 + +- BroadcastExchange + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- 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 -Comet accelerated 98 out of 108 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 108 eligible operators (53%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt index fe3b28849a..6c0981f51f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt @@ -1,68 +1,57 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..21ba0466c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt @@ -1,57 +1,68 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index 7a2832757d..59e112c499 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt index 36b9a8306d..1ce83c5a0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt @@ -1,100 +1,115 @@ -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 +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- 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 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index 0443f5fc55..2d4c85b5a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt @@ -1,101 +1,93 @@ Project -+- BroadcastNestedLoopJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt index 7bae7d5724..a29873fa6e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt @@ -1,88 +1,101 @@ Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- 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.item +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- 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.item -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt index 52d7828b06..0bfc07426a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site + : +- CometNativeScan parquet spark_catalog.default.warehouse +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt index 825b1ed81c..e2f1062058 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt @@ -4,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt index 6e3f37148b..8aa248f6ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -117,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.income_band + : : : +- CometNativeScan parquet spark_catalog.default.promotion : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -162,22 +160,20 @@ CometColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -239,13 +235,13 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt index 059acab385..da58064931 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt @@ -40,20 +40,22 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -160,20 +162,22 @@ CometColumnarToRow : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -244,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt index b1bae81b20..402cc842ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt @@ -1,60 +1,57 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt index 1fc09bd2e3..9a5a06cf91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt @@ -1,52 +1,60 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index caf49b6d52..3d8af2a097 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt @@ -1,82 +1,74 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.ship_mode - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.ship_mode +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.ship_mode + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt index ca39e001b0..d77f6eab96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt @@ -1,70 +1,82 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : +- 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.time_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : +- 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.time_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt index c3c3850224..c7d118c3d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt @@ -3,41 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt index 708d91d578..2ff013c178 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt @@ -3,36 +3,41 @@ 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 - +- 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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt index c122bf3803..d35968a3da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan 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 + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt index c842000e11..adadb2dc4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt @@ -1,49 +1,56 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index 821b101f54..67d364800e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt @@ -9,46 +9,52 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,4 +69,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt index b4fbb455ef..63b1420137 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt @@ -9,42 +9,46 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -59,4 +63,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt index f40fdbe4e3..35091ce142 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt index 655f651a41..59da61ba7a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt @@ -1,39 +1,44 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt index 4abbd20203..1d85f12ca6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt @@ -11,19 +11,18 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,29 +35,31 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- Filter - +- Window - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 52 eligible operators (34%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 56 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt index 6a918ac127..791ae2720b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt @@ -11,18 +11,19 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,26 +36,29 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Window + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 52 eligible operators (34%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt index 427d3518dc..5d11aad358 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt @@ -1,62 +1,63 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt index 25ea15c536..673dbd4ada 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt @@ -1,53 +1,62 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt index 081972705a..7a49cad749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt index 9000d27d1b..c2058aa3a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt @@ -6,67 +6,77 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Filter + : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt index 476c7be954..12b5755eb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt @@ -1,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt index 9f6954be38..33a79ee1f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,55 +42,57 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometSort +- CometExchange +- CometFilter @@ -107,28 +106,25 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,54 +133,56 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt index 22ccc17641..7afcaebec8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt @@ -15,25 +15,28 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -42,21 +45,24 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -65,21 +71,24 @@ CometColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -98,25 +107,28 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -125,21 +137,24 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -148,25 +163,28 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt index d12b8dde24..4b08389eae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt @@ -5,131 +5,138 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt index ed8a9e38ca..5ba92ec55f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt @@ -5,113 +5,131 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange + : +- BroadcastHashJoin + : :- HashAggregate : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin + : :- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt index 65256f2124..c253b8aff8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,16 +41,19 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -66,18 +68,21 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..eec4fb502f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt @@ -14,14 +14,15 @@ TakeOrderedAndProject : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -41,10 +42,11 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery + : : : +- CometColumnarExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -64,10 +66,11 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery + : : +- CometColumnarExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -77,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt index 05bd194c34..add26d22d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt index 012403275a..235c1e92f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt @@ -1,39 +1,44 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt index 257b7f0d77..86b85906aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt @@ -1,56 +1,52 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt index 8e04bbeba7..a240608c19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt @@ -1,52 +1,56 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- 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 - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt index 9d97e27318..5ffd41f4ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt @@ -19,16 +19,15 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -64,16 +63,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -81,7 +84,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,16 +107,20 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -121,7 +128,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_site + : : +- CometNativeScan parquet spark_catalog.default.store : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -131,4 +138,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt index ae6fc19b60..fccbf0a389 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt @@ -19,15 +19,16 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -63,10 +64,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -102,10 +104,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery + : : : : : +- CometColumnarExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -128,4 +131,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt index 0e70ec42fa..aab91b60e9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt index 9708802e79..e6286a9886 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt @@ -1,65 +1,75 @@ -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 +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- Filter + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt index c86e7ba2cb..3fa31193d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt index 9fbb7893b8..4da4d56042 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt @@ -1,34 +1,38 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 30 eligible operators (50%). 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/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index e0024369f8..23598e5776 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt @@ -1,130 +1,133 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 114 eligible operators (63%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt index d325825292..3056740a56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt @@ -1,119 +1,130 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 109 out of 114 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 114 eligible operators (63%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt index 7f8b90a86c..dde36ec726 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt @@ -1,64 +1,56 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- BroadcastExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.reason +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_page + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt index e2383394d1..3797565aed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt @@ -1,56 +1,64 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- BroadcastExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : : +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt index 41081debd9..86dc323961 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt @@ -3,34 +3,30 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt index dfc810b108..e899c49c4e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt @@ -3,30 +3,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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index 638b8865e0..05d07c8d36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt @@ -7,76 +7,77 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometColumnarToRow : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometColumnarToRow : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange +- CometColumnarToRow +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt index ddee139acf..e2b06cf20d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt @@ -7,67 +7,76 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometColumnarToRow : : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometColumnarToRow : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +- BroadcastExchange +- CometColumnarToRow +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt index 0b6c2edaa0..6ceb59316c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt index 825b1ed81c..e2f1062058 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt @@ -4,34 +4,39 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt index 7499c5aafc..9f7d7f87c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt @@ -1,59 +1,51 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt index a1c6b73d37..c0b4c3085d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt @@ -1,51 +1,59 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : : : : : +- BroadcastExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt index 2d15266049..5695ae40ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt index 20df9a8047..874eb62d3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt @@ -1,42 +1,48 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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 - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +HashAggregate ++- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt index 798bfdaf1b..9fe719d3c7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt @@ -6,36 +6,37 @@ CometColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.catalog_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt index e1aefb8382..82e4b4b23c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt @@ -6,32 +6,36 @@ CometColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- ColumnarToRow + : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt index 030031856f..fc4889c2d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt @@ -6,33 +6,29 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt index 4c972848e7..e4611ec552 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt @@ -6,29 +6,33 @@ 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt index c075163849..6059603410 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometNativeScan parquet spark_catalog.default.warehouse +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt index 8aebe6be9a..818dd70bdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt index 2cdc75e15e..f450ef0587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt @@ -1,56 +1,65 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt index 476c7be954..12b5755eb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt @@ -1,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt index 6c2a775097..087450c911 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..9c704a0ed6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt @@ -3,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt index c4138c254a..4b85046c30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt @@ -1,404 +1,394 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt index f56d229b68..2b714db2ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt @@ -1,339 +1,404 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- BroadcastHashJoin + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt index c6478057d9..367fe17ed9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt @@ -1,2779 +1,2750 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt index 83deb45432..9539604b53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -1,2350 +1,2779 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt index eb411c16e8..f74b47bf8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt @@ -1,253 +1,234 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt index b18a444bb9..636e8a5322 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt @@ -1,214 +1,253 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt index 369ec68bb4..79803b0fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt index cd52b2cd12..e768681167 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt @@ -3,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt index fb0e21025e..180582c6a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt @@ -5,27 +5,25 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -33,4 +31,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt index 6cb89465d2..05ee5744ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt @@ -5,25 +5,27 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- BroadcastNestedLoopJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -31,4 +33,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt index 543f666a74..3a2d2bd7db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt @@ -1,187 +1,159 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt index c018673888..d3043a5277 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt @@ -1,159 +1,187 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt index 9926d14ae2..8cc2a333f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt @@ -1,116 +1,109 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt index affbc9ef37..0c5aac8292 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt @@ -1,99 +1,116 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt index 8aebe6be9a..818dd70bdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt index 2cdc75e15e..f450ef0587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt @@ -1,56 +1,65 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt index 7d9e9883d2..195d72cb2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt @@ -5,118 +5,101 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt index 1f6c984b22..a0e30dcb5b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt @@ -5,101 +5,118 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt index 092083e465..33a209d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt index b50b570b4b..a3eb870b39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt @@ -10,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt index 6c4ded1515..ff7ad3b219 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt index 75684a9669..b10cf058e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt @@ -11,31 +11,35 @@ CometColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -43,26 +47,30 @@ CometColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -70,25 +78,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt index 8eb3f067e2..6714125d8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt @@ -23,53 +23,47 @@ TakeOrderedAndProject : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : : +- CometColumnarToRow : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -82,43 +76,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] @@ -140,53 +138,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -199,42 +191,46 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt index f7d3371108..cc2068bae6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt @@ -23,47 +23,53 @@ 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 - : : : +- 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 + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -76,37 +82,43 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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).] @@ -128,47 +140,53 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -181,36 +199,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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt index 65c66a7da8..a6f94c745d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt index de7d26cd06..5234ab1b35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt @@ -10,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt index 63ea8e5ea8..74efb6d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt @@ -1,322 +1,344 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt index 35b86dbff5..85d0f4d4cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt @@ -1,269 +1,322 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Union + : : :- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt index fe3b28849a..6c0981f51f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt @@ -1,68 +1,57 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..21ba0466c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt @@ -1,57 +1,68 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt index 6e3f37148b..8aa248f6ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -117,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.income_band + : : : +- CometNativeScan parquet spark_catalog.default.promotion : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -162,22 +160,20 @@ CometColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -239,13 +235,13 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt index 059acab385..da58064931 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt @@ -40,20 +40,22 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -160,20 +162,22 @@ CometColumnarToRow : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -244,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt index 0d338d5831..87a6683816 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt @@ -5,346 +5,294 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit - +- Sort - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 127 out of 285 eligible operators (44%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt index b96ce25ae1..d40b7fd814 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt @@ -5,294 +5,346 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- WindowGroupLimit + +- Sort + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 127 out of 285 eligible operators (44%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt index d13ccdd19f..1de2522a2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt @@ -14,19 +14,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,30 +39,32 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -74,19 +75,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -100,30 +100,32 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -134,19 +136,18 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,29 +161,31 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 156 eligible operators (34%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 168 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index 191d0ef18a..be77e5eca2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -14,18 +14,19 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,27 +40,30 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- WindowGroupLimit + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -70,18 +74,19 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -95,27 +100,30 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- WindowGroupLimit + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -126,18 +134,19 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -151,26 +160,29 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 156 eligible operators (34%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt index 081972705a..7a49cad749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt index 9000d27d1b..c2058aa3a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt @@ -6,67 +6,77 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Filter + : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt index 476c7be954..12b5755eb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt @@ -1,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt index 9f6954be38..33a79ee1f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,55 +42,57 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometSort +- CometExchange +- CometFilter @@ -107,28 +106,25 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,54 +133,56 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt index 22ccc17641..7afcaebec8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt @@ -15,25 +15,28 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -42,21 +45,24 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -65,21 +71,24 @@ CometColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -98,25 +107,28 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -125,21 +137,24 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -148,25 +163,28 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt index 81b485e6b3..e18882dfff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt @@ -9,132 +9,139 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -144,132 +151,139 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -279,131 +293,138 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt index b4318d03e2..0fef4fef27 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt @@ -9,114 +9,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange + : : +- BroadcastHashJoin + : : :- HashAggregate : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -126,114 +144,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange + : : +- BroadcastHashJoin + : : :- HashAggregate : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -243,113 +279,131 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange + : +- BroadcastHashJoin + : :- HashAggregate : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin + : :- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt index 65256f2124..c253b8aff8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,16 +41,19 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -66,18 +68,21 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..eec4fb502f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt @@ -14,14 +14,15 @@ TakeOrderedAndProject : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -41,10 +42,11 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery + : : : +- CometColumnarExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -64,10 +66,11 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery + : : +- CometColumnarExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -77,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt index f752943c47..b66eb1ce18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt @@ -22,16 +22,15 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -67,16 +66,20 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -84,7 +87,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -107,16 +110,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -124,7 +131,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_site + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -154,16 +161,15 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -199,16 +205,20 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -216,7 +226,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -239,16 +249,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -256,7 +270,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_site + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -286,16 +300,15 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -331,16 +344,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -348,7 +365,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -371,16 +388,20 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -388,7 +409,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_site + : : +- CometNativeScan parquet spark_catalog.default.store : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -398,4 +419,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt index 0d6844d93c..c2f4c7cc7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt @@ -22,15 +22,16 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -66,10 +67,11 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -105,10 +107,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -151,15 +154,16 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -195,10 +199,11 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -234,10 +239,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -280,15 +286,16 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -324,10 +331,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -363,10 +371,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery + : : : : : +- CometColumnarExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -389,4 +398,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt index a5b37c422c..4374f0aea2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt @@ -5,97 +5,83 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt index 46c47555a8..f7d1e34134 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt @@ -5,83 +5,97 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt index 9ab057d467..ca5d077909 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt @@ -5,33 +5,29 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt index 3dbaf2e346..e6bdee919f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt @@ -5,29 +5,33 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt index 8aebe6be9a..818dd70bdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt index 2cdc75e15e..f450ef0587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt @@ -1,56 +1,65 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 476c7be954..12b5755eb7 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,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). 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/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt index 6c2a775097..087450c911 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 3f41c97ff5..9c704a0ed6 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,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). 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/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt index e44d85bdc2..6d6eda80ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt @@ -1,408 +1,414 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 337 eligible operators (37%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 343 out of 387 eligible operators (88%). Final plan contains 25 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt index b32dd95954..a02b0fda3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt @@ -1,343 +1,408 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- ReusedSubquery +TakeOrderedAndProject ++- BroadcastHashJoin + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- 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 + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- 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 + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- 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 + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : +- 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 -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 337 eligible operators (37%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt index c6478057d9..367fe17ed9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt @@ -1,2779 +1,2750 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt index 83deb45432..9539604b53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt @@ -1,2350 +1,2779 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt index eb411c16e8..f74b47bf8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt @@ -1,253 +1,234 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt index b18a444bb9..636e8a5322 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt @@ -1,214 +1,253 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt index 369ec68bb4..79803b0fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 cd52b2cd12..e768681167 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,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). 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/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt index fb0e21025e..180582c6a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt @@ -5,27 +5,25 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -33,4 +31,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt index 6cb89465d2..05ee5744ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt @@ -5,25 +5,27 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- BroadcastNestedLoopJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -31,4 +33,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt index 543f666a74..3a2d2bd7db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt @@ -1,187 +1,159 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt index c018673888..d3043a5277 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt @@ -1,159 +1,187 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt index 9926d14ae2..8cc2a333f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt @@ -1,116 +1,109 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt index affbc9ef37..0c5aac8292 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt @@ -1,99 +1,116 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt index 8aebe6be9a..818dd70bdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt index 2cdc75e15e..f450ef0587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt @@ -1,56 +1,65 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt index 7d9e9883d2..195d72cb2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt @@ -5,118 +5,101 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 1f6c984b22..a0e30dcb5b 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,101 +5,118 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 99 eligible operators (45%). 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/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt index 092083e465..33a209d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 b50b570b4b..a3eb870b39 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,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). 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/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt index 6c4ded1515..ff7ad3b219 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt index 75684a9669..b10cf058e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt @@ -11,31 +11,35 @@ 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -43,26 +47,30 @@ CometColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -70,25 +78,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt index 8eb3f067e2..6714125d8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt @@ -23,53 +23,47 @@ TakeOrderedAndProject : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : : +- CometColumnarToRow : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -82,43 +76,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] @@ -140,53 +138,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -199,42 +191,46 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 f7d3371108..cc2068bae6 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,47 +23,53 @@ 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 - : : : +- 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 + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -76,37 +82,43 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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).] @@ -128,47 +140,53 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -181,36 +199,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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt index 65c66a7da8..a6f94c745d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 de7d26cd06..5234ab1b35 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,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). 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/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt index 63ea8e5ea8..74efb6d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt @@ -1,322 +1,344 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 35b86dbff5..85d0f4d4cb 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,269 +1,322 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Union + : : :- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt index ee120ac920..75418138a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt @@ -1,70 +1,73 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 60 eligible operators (65%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 60 out of 66 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt index 2d049f149f..2191f4ba49 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt @@ -1,59 +1,70 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- 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 - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- 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 + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- 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 + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 60 eligible operators (65%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt index 6e3f37148b..8aa248f6ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -117,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.income_band + : : : +- CometNativeScan parquet spark_catalog.default.promotion : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -162,22 +160,20 @@ CometColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -239,13 +235,13 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 059acab385..da58064931 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 @@ -40,20 +40,22 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -160,20 +162,22 @@ CometColumnarToRow : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -244,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt index 0d338d5831..87a6683816 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt @@ -5,346 +5,294 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit - +- Sort - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 127 out of 285 eligible operators (44%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt index b96ce25ae1..d40b7fd814 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt @@ -5,294 +5,346 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- WindowGroupLimit + +- Sort + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 127 out of 285 eligible operators (44%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt index d13ccdd19f..1de2522a2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt @@ -14,19 +14,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,30 +39,32 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -74,19 +75,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -100,30 +100,32 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -134,19 +136,18 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,29 +161,31 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 156 eligible operators (34%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 168 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 191d0ef18a..be77e5eca2 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 @@ -14,18 +14,19 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,27 +40,30 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- WindowGroupLimit + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -70,18 +74,19 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -95,27 +100,30 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- WindowGroupLimit + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -126,18 +134,19 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -151,26 +160,29 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 156 eligible operators (34%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt index 081972705a..7a49cad749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt index 9000d27d1b..c2058aa3a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt @@ -6,67 +6,77 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Filter + : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 476c7be954..12b5755eb7 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,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). 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/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt index 9f6954be38..33a79ee1f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,55 +42,57 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometSort +- CometExchange +- CometFilter @@ -107,28 +106,25 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,54 +133,56 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 22ccc17641..7afcaebec8 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 @@ -15,25 +15,28 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -42,21 +45,24 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -65,21 +71,24 @@ CometColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -98,25 +107,28 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -125,21 +137,24 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -148,25 +163,28 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt index 81b485e6b3..e18882dfff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt @@ -9,132 +9,139 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -144,132 +151,139 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -279,131 +293,138 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 b4318d03e2..0fef4fef27 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,114 +9,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange + : : +- BroadcastHashJoin + : : :- HashAggregate : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -126,114 +144,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange + : : +- BroadcastHashJoin + : : :- HashAggregate : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -243,113 +279,131 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange + : +- BroadcastHashJoin + : :- HashAggregate : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin + : :- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt index 65256f2124..c253b8aff8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,16 +41,19 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -66,18 +68,21 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 9d5ba53be1..eec4fb502f 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 @@ -14,14 +14,15 @@ TakeOrderedAndProject : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -41,10 +42,11 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery + : : : +- CometColumnarExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -64,10 +66,11 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery + : : +- CometColumnarExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -77,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt index f752943c47..b66eb1ce18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt @@ -22,16 +22,15 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -67,16 +66,20 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -84,7 +87,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -107,16 +110,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -124,7 +131,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_site + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -154,16 +161,15 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -199,16 +205,20 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -216,7 +226,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -239,16 +249,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -256,7 +270,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_site + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -286,16 +300,15 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -331,16 +344,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -348,7 +365,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -371,16 +388,20 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -388,7 +409,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_site + : : +- CometNativeScan parquet spark_catalog.default.store : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -398,4 +419,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 0d6844d93c..c2f4c7cc7c 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 @@ -22,15 +22,16 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -66,10 +67,11 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -105,10 +107,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -151,15 +154,16 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -195,10 +199,11 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -234,10 +239,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -280,15 +286,16 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -324,10 +331,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -363,10 +371,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery + : : : : : +- CometColumnarExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -389,4 +398,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt index a5b37c422c..4374f0aea2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt @@ -5,97 +5,83 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 46c47555a8..f7d1e34134 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,83 +5,97 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 81 eligible operators (44%). 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/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt index 9ab057d467..ca5d077909 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt @@ -5,33 +5,29 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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 3dbaf2e346..e6bdee919f 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,29 +5,33 @@ 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt index 8aebe6be9a..818dd70bdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt index 2cdc75e15e..f450ef0587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt @@ -1,56 +1,65 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt index 476c7be954..12b5755eb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt @@ -1,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). 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/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt index 6c2a775097..087450c911 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt index 3f41c97ff5..9c704a0ed6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt @@ -3,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). 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/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index c4138c254a..4b85046c30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt @@ -1,404 +1,394 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt index f56d229b68..2b714db2ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt @@ -1,339 +1,404 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- BroadcastHashJoin + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt index c6478057d9..367fe17ed9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt @@ -1,2779 +1,2750 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt index 83deb45432..9539604b53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt @@ -1,2350 +1,2779 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Union + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : :- Filter + : : : +- ReusedSubquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometColumnarExchange + : : : : : +- HashAggregate + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- BroadcastHashJoin + : : : : : : :- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Filter + : : : : : : : : +- ColumnarToRow + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- ReusedSubquery + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Filter + : : +- ReusedSubquery + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometColumnarExchange + : : : : +- HashAggregate + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : +- BroadcastExchange + : : : : : +- BroadcastHashJoin + : : : : : :- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Filter + : +- ReusedSubquery + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt index eb411c16e8..f74b47bf8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt @@ -1,253 +1,234 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt index b18a444bb9..636e8a5322 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt @@ -1,214 +1,253 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt index 369ec68bb4..79803b0fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt index cd52b2cd12..e768681167 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt @@ -3,29 +3,33 @@ 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 27 eligible operators (44%). 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/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt index fb0e21025e..180582c6a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt @@ -5,27 +5,25 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -33,4 +31,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt index 6cb89465d2..05ee5744ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt @@ -5,25 +5,27 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- BroadcastNestedLoopJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -31,4 +33,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt index 543f666a74..3a2d2bd7db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt @@ -1,187 +1,159 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt index c018673888..d3043a5277 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt @@ -1,159 +1,187 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt index 9926d14ae2..8cc2a333f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt @@ -1,116 +1,109 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt index affbc9ef37..0c5aac8292 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt @@ -1,99 +1,116 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt index 9e4596c12c..b63d5c384c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt index 3d9d23d2cc..494d98d409 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt @@ -1,41 +1,47 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometColumnarExchange + +- Project + +- BroadcastHashJoin + :- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 8935b6cc13..4446b1d56b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt index a6f33d6f7e..68b088493c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt @@ -10,42 +10,46 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange + : : :- BroadcastHashJoin + : : : :- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Project + : : : : +- BroadcastHashJoin + : : : : :- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt index 8aebe6be9a..818dd70bdf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt index 2cdc75e15e..f450ef0587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt @@ -1,56 +1,65 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics +TakeOrderedAndProject ++- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Union + : : :- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt index 7d9e9883d2..195d72cb2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt @@ -5,118 +5,101 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt index 1f6c984b22..a0e30dcb5b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt @@ -5,101 +5,118 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 99 eligible operators (45%). 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/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt index 092083e465..33a209d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt index b50b570b4b..a3eb870b39 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt @@ -10,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). 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/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index 6c4ded1515..ff7ad3b219 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ 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 - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt index 75684a9669..b10cf058e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt @@ -11,31 +11,35 @@ 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -43,26 +47,30 @@ CometColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -70,25 +78,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastExchange + : : +- Project + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index 8eb3f067e2..6714125d8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -23,53 +23,47 @@ TakeOrderedAndProject : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : : +- CometColumnarToRow : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -82,43 +76,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] @@ -140,53 +138,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -199,42 +191,46 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt index f7d3371108..cc2068bae6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt @@ -23,47 +23,53 @@ 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 - : : : +- 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 + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -76,37 +82,43 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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).] @@ -128,47 +140,53 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -181,36 +199,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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt index 65c66a7da8..a6f94c745d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt index de7d26cd06..5234ab1b35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt @@ -10,96 +10,111 @@ 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 - : : +- 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 + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- 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 + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - +- 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 + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 97 eligible operators (37%). 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/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index 63ea8e5ea8..74efb6d851 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt @@ -1,322 +1,344 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt index 35b86dbff5..85d0f4d4cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt @@ -1,269 +1,322 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Union + : : : : :- Project + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- BroadcastExchange + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Union + : : : :- Project + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Union + : : :- Project + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- Project + : : +- BroadcastHashJoin + : : :- BroadcastExchange + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt index fe3b28849a..6c0981f51f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt @@ -1,68 +1,57 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt index f5b69fc6cf..21ba0466c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt @@ -1,57 +1,68 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +TakeOrderedAndProject ++- Filter + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt index 6e3f37148b..8aa248f6ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -117,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.income_band + : : : +- CometNativeScan parquet spark_catalog.default.promotion : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -162,22 +160,20 @@ CometColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -239,13 +235,13 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band + : : +- CometNativeScan parquet spark_catalog.default.promotion : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt index 059acab385..da58064931 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt @@ -40,20 +40,22 @@ CometColumnarToRow : : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -160,20 +162,22 @@ CometColumnarToRow : : : : : : : : : : : : : : :- 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 + : : : : : : : : : : : : : : : : +- CometColumnarExchange + : : : : : : : : : : : : : : : : +- Project + : : : : : : : : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : : : : : : : : :- BroadcastExchange + : : : : : : : : : : : : : : : : : +- Filter + : : : : : : : : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -244,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt index a2ddc934bb..2fd6aa960b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt @@ -3,345 +3,292 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 127 out of 282 eligible operators (45%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt index 1735511c2a..ab9cfad7ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt @@ -3,292 +3,345 @@ 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 - +- CometExchange - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometColumnarExchange + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 127 out of 282 eligible operators (45%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt index 1a13c28a4f..527b3ae2d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt @@ -14,19 +14,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,30 +38,32 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- Filter - : +- Window - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -73,19 +74,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -98,30 +98,32 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- Filter - : +- Window - : +- Sort - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -132,19 +134,18 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -157,29 +158,31 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- Filter - +- Window - +- Sort - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 153 eligible operators (35%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 165 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt index a677fed73a..b8c9813039 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt @@ -14,18 +14,19 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -38,27 +39,30 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Window + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -69,18 +73,19 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -93,27 +98,30 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Window + : +- Sort + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -124,18 +132,19 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -148,26 +157,29 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Window + +- Sort + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 153 eligible operators (66%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 153 eligible operators (35%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt index 081972705a..7a49cad749 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt index 9000d27d1b..c2058aa3a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt @@ -6,67 +6,77 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Project + : : : : : +- BroadcastHashJoin + : : : : : :- Project + : : : : : : +- BroadcastHashJoin + : : : : : : :- Project + : : : : : : : +- BroadcastHashJoin + : : : : : : : :- Project + : : : : : : : : +- BroadcastHashJoin + : : : : : : : : :- Project + : : : : : : : : : +- BroadcastHashJoin + : : : : : : : : : :- Filter + : : : : : : : : : : +- ColumnarToRow + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : : : +- BroadcastExchange + : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometFilter + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt index 0d59f7b833..274047acbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt index 476c7be954..12b5755eb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt @@ -1,90 +1,105 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- BroadcastHashJoin + : : :- Filter + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- BroadcastExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- BroadcastExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Filter + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : +- BroadcastExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- Filter + : +- ColumnarToRow + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- ReusedSubquery + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 85 eligible operators (32%). 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/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt index 9f6954be38..33a79ee1f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,55 +42,57 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometSort +- CometExchange +- CometFilter @@ -107,28 +106,25 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,54 +133,56 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_returns + +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt index 22ccc17641..7afcaebec8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt @@ -15,25 +15,28 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -42,21 +45,24 @@ CometColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometColumnarExchange + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -65,21 +71,24 @@ CometColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -98,25 +107,28 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -125,21 +137,24 @@ CometColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometColumnarExchange + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -148,25 +163,28 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index 81b485e6b3..e18882dfff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt @@ -9,132 +9,139 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -144,132 +151,139 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -279,131 +293,138 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt index b4318d03e2..0fef4fef27 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt @@ -9,114 +9,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange + : : +- BroadcastHashJoin + : : :- HashAggregate : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -126,114 +144,132 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange + : : +- BroadcastHashJoin + : : :- HashAggregate : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- Project + : : : : +- BroadcastHashJoin + : : : : :- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- CometColumnarToRow + : : : +- CometColumnarExchange + : : : +- HashAggregate + : : : +- Project + : : : +- BroadcastHashJoin + : : : :- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- ReusedSubquery + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -243,113 +279,131 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange + : +- BroadcastHashJoin + : :- HashAggregate : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- BroadcastHashJoin + : : : :- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin + : :- BroadcastExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- BroadcastHashJoin + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- BroadcastExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt index 65256f2124..c253b8aff8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,16 +41,19 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -66,18 +68,21 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt index 9d5ba53be1..eec4fb502f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt @@ -14,14 +14,15 @@ TakeOrderedAndProject : : : +- 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 + : : : : +- CometColumnarExchange + : : : : +- Filter + : : : : +- ColumnarToRow + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -41,10 +42,11 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery + : : : +- CometColumnarExchange + : : : +- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -64,10 +66,11 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery + : : +- CometColumnarExchange + : : +- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -77,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt index f752943c47..b66eb1ce18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt @@ -22,16 +22,15 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -67,16 +66,20 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -84,7 +87,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -107,16 +110,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -124,7 +131,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_site + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -154,16 +161,15 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -199,16 +205,20 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -216,7 +226,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : : +- CometNativeScan parquet spark_catalog.default.store : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -239,16 +249,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -256,7 +270,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_site + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -286,16 +300,15 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -331,16 +344,20 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -348,7 +365,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : : : +- CometNativeScan parquet spark_catalog.default.store : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -371,16 +388,20 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -388,7 +409,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_site + : : +- CometNativeScan parquet spark_catalog.default.store : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -398,4 +419,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt index 0d6844d93c..c2f4c7cc7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt @@ -22,15 +22,16 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -66,10 +67,11 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -105,10 +107,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -151,15 +154,16 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -195,10 +199,11 @@ CometColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometColumnarExchange + : : : : : : : +- Filter + : : : : : : : +- ColumnarToRow + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -234,10 +239,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -280,15 +286,16 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -324,10 +331,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery + : : : : : : +- CometColumnarExchange + : : : : : : +- Filter + : : : : : : +- ColumnarToRow + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -363,10 +371,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery + : : : : : +- CometColumnarExchange + : : : : : +- Filter + : : : : : +- ColumnarToRow + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -389,4 +398,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt index a5b37c422c..4374f0aea2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt @@ -5,97 +5,83 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt index 46c47555a8..f7d1e34134 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt @@ -5,83 +5,97 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- Filter + : : : +- ColumnarToRow + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 81 eligible operators (44%). 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/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt index 9ab057d467..ca5d077909 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt @@ -5,33 +5,29 @@ 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 - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt index 3dbaf2e346..e6bdee919f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt @@ -5,29 +5,33 @@ 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Filter + : : +- ColumnarToRow + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala index 859c53a9fb..50cbfd76f3 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala @@ -66,11 +66,7 @@ class ParquetReadFromFakeHadoopFsSuite extends CometTestBase with AdaptiveSparkP p } assert(scans.size == 1) - // Access selected partitions through originalPlan (FileSourceScanExec) - val selectedPartitions = scans.head.originalPlan.selectedPartitions - assert( - selectedPartitions.head.files.head.getPath.toString - .startsWith(FakeHDFSFileSystem.PREFIX)) + assert(scans.head.relation.inputFiles.head.startsWith(FakeHDFSFileSystem.PREFIX)) } // This test fails for 'hdfs' but succeeds for 'open-dal'. 'hdfs' requires this fix From b32660e421c37d0435b9ceb34341c88385329776 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Feb 2026 16:49:51 -0500 Subject: [PATCH 06/38] make format --- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index d6ddca0d2b..1c9f240242 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -22,7 +22,7 @@ package org.apache.spark.sql.comet.shims import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BasePredicate, BoundReference, Expression, FileSourceConstantMetadataAttribute, Literal, Predicate} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, FileSourceConstantMetadataAttribute, Literal, Predicate} import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil, ScalarSubquery, ScanFileListing} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions From ee0806ea4efff3530a20049cfa50441de6032839 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Feb 2026 17:45:16 -0500 Subject: [PATCH 07/38] Update CometScanRuleSuite. --- .../scala/org/apache/comet/rules/CometScanRuleSuite.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala index a349ab2b93..fa1030e420 100644 --- a/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala @@ -90,7 +90,12 @@ class CometScanRuleSuite extends CometTestBase { if (cometEnabled) { assert(countOperators(transformedPlan, classOf[FileSourceScanExec]) == 0) - assert(countOperators(transformedPlan, classOf[CometScanExec]) == 1) + if (CometConf.COMET_NATIVE_SCAN_IMPL.get() != CometConf.SCAN_NATIVE_DATAFUSION) { + assert(countOperators(transformedPlan, classOf[CometScanExec]) == 1) + } else { + // CometNativeScanExec does not get wrapped in CometScanExec + assert(countOperators(transformedPlan, classOf[CometNativeScanExec]) == 1) + } } else { assert(countOperators(transformedPlan, classOf[FileSourceScanExec]) == 1) assert(countOperators(transformedPlan, classOf[CometScanExec]) == 0) From d44404653f2da389bdbb4f20f82b2f113d3951fc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 11:54:00 -0500 Subject: [PATCH 08/38] Update the DPP config for Comet. --- .../scala/org/apache/comet/CometConf.scala | 12 +++ .../apache/comet/rules/CometScanRule.scala | 92 ++++++++++++------- 2 files changed, 69 insertions(+), 35 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index c0b526e0a3..81e2ce2ee7 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -137,6 +137,18 @@ object CometConf extends ShimCometConf { .checkValues(Set(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT, SCAN_AUTO)) .createWithEnvVarOrDefault("COMET_PARQUET_SCAN_IMPL", SCAN_AUTO) + val COMET_DPP_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.scan.dpp.enabled") + .category(CATEGORY_SCAN) + .doc( + "Whether to enable Dynamic Partition Pruning (DPP) for native_datafusion scans. " + + "When enabled, queries with DPP use CometNativeScanExec which supports runtime " + + "partition filtering. When disabled, DPP queries fall back to Spark. " + + "This config only affects native_datafusion scans; other scan modes always " + + "fall back to Spark for DPP queries.") + .booleanConf + .createWithDefault(true) + val COMET_ICEBERG_NATIVE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.icebergNative.enabled") .category(CATEGORY_SCAN) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 0a860e429f..cb4aef4b55 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -171,21 +171,33 @@ case class CometScanRule(session: SparkSession) case SCAN_AUTO => // TODO add support for native_datafusion in the future if (hasDPP) { - return withInfo(scanExec, "Dynamic Partition Pruning is not supported") + return withInfo( + scanExec, + "Dynamic Partition Pruning is not supported with " + + "spark.comet.scan.impl=auto.") } nativeIcebergCompatScan(session, scanExec, r, hadoopConf) .getOrElse(scanExec) case SCAN_NATIVE_DATAFUSION => - // native_datafusion supports DPP + // native_datafusion supports DPP via CometNativeScanExec when enabled + if (hasDPP && !CometConf.COMET_DPP_ENABLED.get()) { + return withInfo(scanExec, "Dynamic Partition Pruning is not enabled.") + } nativeDataFusionScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) case SCAN_NATIVE_ICEBERG_COMPAT => if (hasDPP) { - return withInfo(scanExec, "Dynamic Partition Pruning is not supported") + return withInfo( + scanExec, + "Dynamic Partition Pruning is not supported with " + + "spark.comet.scan.impl=native_iceberg_compat.") } nativeIcebergCompatScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) case SCAN_NATIVE_COMET => if (hasDPP) { - return withInfo(scanExec, "Dynamic Partition Pruning is not supported") + return withInfo( + scanExec, + "Dynamic Partition Pruning is not supported with " + + "spark.comet.scan.impl=native_comet.") } nativeCometScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) } @@ -649,40 +661,50 @@ case class CometScanRule(session: SparkSession) !hasUnsupportedDeletes } - // Check that all DPP subqueries use InSubqueryExec which we know how to handle. - // Future Spark versions might introduce new subquery types we haven't tested. + // Check DPP support - controlled by COMET_DPP_ENABLED config + val hasDppFilters = scanExec.runtimeFilters.exists { + case DynamicPruningExpression(_) => true + case _ => false + } val dppSubqueriesSupported = { - val unsupportedSubqueries = scanExec.runtimeFilters.collect { - case DynamicPruningExpression(e) if !e.isInstanceOf[InSubqueryExec] => - e.getClass.getSimpleName - } - // Check for multi-index DPP which we don't support yet. - // SPARK-46946 changed SubqueryAdaptiveBroadcastExec from index: Int to indices: Seq[Int] - // as a preparatory refactor for future features (Null Safe Equality DPP, multiple - // equality predicates). Currently indices always has one element, but future Spark - // versions might use multiple indices. - val multiIndexDpp = scanExec.runtimeFilters.exists { - case DynamicPruningExpression(e: InSubqueryExec) => - e.plan match { - case sab: SubqueryAdaptiveBroadcastExec => - getSubqueryBroadcastIndices(sab).length > 1 - case _ => false - } - case _ => false - } - if (unsupportedSubqueries.nonEmpty) { - fallbackReasons += - s"Unsupported DPP subquery types: ${unsupportedSubqueries.mkString(", ")}. " + - "CometIcebergNativeScanExec only supports InSubqueryExec for DPP" - false - } else if (multiIndexDpp) { - // See SPARK-46946 for context on multi-index DPP - fallbackReasons += - "Multi-index DPP (indices.length > 1) is not yet supported. " + - "See SPARK-46946 for context." + if (hasDppFilters && !CometConf.COMET_DPP_ENABLED.get()) { + fallbackReasons += "Dynamic Partition Pruning is disabled" false } else { - true + // Check that all DPP subqueries use InSubqueryExec which we know how to handle. + // Future Spark versions might introduce new subquery types we haven't tested. + val unsupportedSubqueries = scanExec.runtimeFilters.collect { + case DynamicPruningExpression(e) if !e.isInstanceOf[InSubqueryExec] => + e.getClass.getSimpleName + } + // Check for multi-index DPP which we don't support yet. + // SPARK-46946 changed SubqueryAdaptiveBroadcastExec from index: Int to + // indices: Seq[Int] as a preparatory refactor for future features (Null Safe + // Equality DPP, multiple equality predicates). Currently indices always has + // one element, but future Spark versions might use multiple indices. + val multiIndexDpp = scanExec.runtimeFilters.exists { + case DynamicPruningExpression(e: InSubqueryExec) => + e.plan match { + case sab: SubqueryAdaptiveBroadcastExec => + getSubqueryBroadcastIndices(sab).length > 1 + case _ => false + } + case _ => false + } + if (unsupportedSubqueries.nonEmpty) { + fallbackReasons += + s"Unsupported DPP subquery types: ${unsupportedSubqueries.mkString(", ")}. " + + "CometIcebergNativeScanExec only supports InSubqueryExec for DPP" + false + } else if (multiIndexDpp) { + // See SPARK-46946 for context on multi-index DPP + fallbackReasons += + "Multi-index DPP (indices.length > 1) is not yet supported. " + + "See SPARK-46946 for context." + false + } else { + true + } } } From 59d7b41726a2da2b05fbcabba281ba41cba6ea9e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 12:04:59 -0500 Subject: [PATCH 09/38] Fix after upmerge. --- .../main/scala/org/apache/comet/rules/CometScanRule.scala | 8 -------- 1 file changed, 8 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 1163ab85ff..8d61e72070 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -189,14 +189,6 @@ case class CometScanRule(session: SparkSession) "spark.comet.scan.impl=native_iceberg_compat.") } nativeIcebergCompatScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) - case SCAN_NATIVE_COMET => - if (hasDPP) { - return withInfo( - scanExec, - "Dynamic Partition Pruning is not supported with " + - "spark.comet.scan.impl=native_comet.") - } - nativeCometScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) } case _ => From 259fc99b28e6471cac2651a58cc6a4b4eaa0cdbc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 12:30:55 -0500 Subject: [PATCH 10/38] Update plans. --- .../q1.native_iceberg_compat/extended.txt | 4 +- .../q10.native_iceberg_compat/extended.txt | 6 +- .../q11.native_iceberg_compat/extended.txt | 8 +- .../q12.native_iceberg_compat/extended.txt | 2 +- .../q13.native_iceberg_compat/extended.txt | 2 +- .../q14a.native_iceberg_compat/extended.txt | 48 ++-- .../q14b.native_iceberg_compat/extended.txt | 34 +-- .../q15.native_iceberg_compat/extended.txt | 2 +- .../q17.native_iceberg_compat/extended.txt | 6 +- .../q18.native_iceberg_compat/extended.txt | 2 +- .../q20.native_iceberg_compat/extended.txt | 2 +- .../q21.native_iceberg_compat/extended.txt | 2 +- .../q22.native_iceberg_compat/extended.txt | 2 +- .../q23a.native_iceberg_compat/extended.txt | 10 +- .../q23b.native_iceberg_compat/extended.txt | 10 +- .../q25.native_iceberg_compat/extended.txt | 6 +- .../q26.native_iceberg_compat/extended.txt | 2 +- .../q27.native_iceberg_compat/extended.txt | 2 +- .../q29.native_iceberg_compat/extended.txt | 6 +- .../q30.native_iceberg_compat/extended.txt | 4 +- .../q31.native_iceberg_compat/extended.txt | 12 +- .../q32.native_iceberg_compat/extended.txt | 4 +- .../q33.native_iceberg_compat/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 2 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q36.native_iceberg_compat/extended.txt | 2 +- .../q37.native_iceberg_compat/extended.txt | 2 +- .../q38.native_iceberg_compat/extended.txt | 6 +- .../q39a.native_iceberg_compat/extended.txt | 4 +- .../q39b.native_iceberg_compat/extended.txt | 4 +- .../q4.native_iceberg_compat/extended.txt | 12 +- .../q40.native_iceberg_compat/extended.txt | 2 +- .../q45.native_iceberg_compat/extended.txt | 2 +- .../q46.native_iceberg_compat/extended.txt | 2 +- .../q47.native_iceberg_compat/extended.txt | 6 +- .../q48.native_iceberg_compat/extended.txt | 2 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q5.native_iceberg_compat/extended.txt | 12 +- .../q50.native_iceberg_compat/extended.txt | 2 +- .../q51.native_iceberg_compat/extended.txt | 4 +- .../q53.native_iceberg_compat/extended.txt | 2 +- .../q54.native_iceberg_compat/extended.txt | 6 +- .../q56.native_iceberg_compat/extended.txt | 6 +- .../q57.native_iceberg_compat/extended.txt | 6 +- .../q58.native_iceberg_compat/extended.txt | 6 +- .../q6.native_iceberg_compat/extended.txt | 2 +- .../q60.native_iceberg_compat/extended.txt | 6 +- .../q61.native_iceberg_compat/extended.txt | 4 +- .../q63.native_iceberg_compat/extended.txt | 2 +- .../q64.native_iceberg_compat/extended.txt | 4 +- .../q65.native_iceberg_compat/extended.txt | 4 +- .../q66.native_iceberg_compat/extended.txt | 4 +- .../q67.native_iceberg_compat/extended.txt | 2 +- .../q68.native_iceberg_compat/extended.txt | 2 +- .../q69.native_iceberg_compat/extended.txt | 6 +- .../q7.native_iceberg_compat/extended.txt | 2 +- .../q70.native_iceberg_compat/extended.txt | 4 +- .../q71.native_iceberg_compat/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 2 +- .../q73.native_iceberg_compat/extended.txt | 2 +- .../q74.native_iceberg_compat/extended.txt | 8 +- .../q75.native_iceberg_compat/extended.txt | 12 +- .../q77.native_iceberg_compat/extended.txt | 12 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q79.native_iceberg_compat/extended.txt | 2 +- .../q8.native_iceberg_compat/extended.txt | 2 +- .../q80.native_iceberg_compat/extended.txt | 6 +- .../q81.native_iceberg_compat/extended.txt | 4 +- .../q82.native_iceberg_compat/extended.txt | 2 +- .../q83.native_iceberg_compat/extended.txt | 6 +- .../q85.native_iceberg_compat/extended.txt | 2 +- .../q86.native_iceberg_compat/extended.txt | 2 +- .../q87.native_iceberg_compat/extended.txt | 6 +- .../q89.native_iceberg_compat/extended.txt | 2 +- .../q91.native_iceberg_compat/extended.txt | 2 +- .../q92.native_iceberg_compat/extended.txt | 4 +- .../q97.native_iceberg_compat/extended.txt | 4 +- .../q98.native_iceberg_compat/extended.txt | 2 +- .../q1.native_iceberg_compat/extended.txt | 4 +- .../q10.native_iceberg_compat/extended.txt | 6 +- .../q11.native_iceberg_compat/extended.txt | 8 +- .../q12.native_iceberg_compat/extended.txt | 2 +- .../q13.native_iceberg_compat/extended.txt | 2 +- .../q14a.native_iceberg_compat/extended.txt | 48 ++-- .../q14b.native_iceberg_compat/extended.txt | 34 +-- .../q15.native_iceberg_compat/extended.txt | 2 +- .../q17.native_iceberg_compat/extended.txt | 6 +- .../q18.native_iceberg_compat/extended.txt | 2 +- .../q20.native_iceberg_compat/extended.txt | 2 +- .../q21.native_iceberg_compat/extended.txt | 2 +- .../q22.native_iceberg_compat/extended.txt | 2 +- .../q23a.native_iceberg_compat/extended.txt | 10 +- .../q23b.native_iceberg_compat/extended.txt | 10 +- .../q25.native_iceberg_compat/extended.txt | 6 +- .../q26.native_iceberg_compat/extended.txt | 2 +- .../q27.native_iceberg_compat/extended.txt | 2 +- .../q29.native_iceberg_compat/extended.txt | 6 +- .../q30.native_iceberg_compat/extended.txt | 4 +- .../q31.native_iceberg_compat/extended.txt | 12 +- .../q32.native_iceberg_compat/extended.txt | 4 +- .../q33.native_iceberg_compat/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 2 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q36.native_iceberg_compat/extended.txt | 2 +- .../q37.native_iceberg_compat/extended.txt | 2 +- .../q38.native_iceberg_compat/extended.txt | 6 +- .../q39a.native_iceberg_compat/extended.txt | 4 +- .../q39b.native_iceberg_compat/extended.txt | 4 +- .../q4.native_iceberg_compat/extended.txt | 12 +- .../q40.native_iceberg_compat/extended.txt | 2 +- .../q45.native_iceberg_compat/extended.txt | 2 +- .../q46.native_iceberg_compat/extended.txt | 2 +- .../q47.native_iceberg_compat/extended.txt | 6 +- .../q48.native_iceberg_compat/extended.txt | 2 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q5.native_iceberg_compat/extended.txt | 12 +- .../q50.native_iceberg_compat/extended.txt | 2 +- .../q51.native_iceberg_compat/extended.txt | 4 +- .../q53.native_iceberg_compat/extended.txt | 2 +- .../q54.native_iceberg_compat/extended.txt | 6 +- .../q56.native_iceberg_compat/extended.txt | 6 +- .../q57.native_iceberg_compat/extended.txt | 6 +- .../q58.native_iceberg_compat/extended.txt | 6 +- .../q6.native_iceberg_compat/extended.txt | 2 +- .../q60.native_iceberg_compat/extended.txt | 6 +- .../q61.native_iceberg_compat/extended.txt | 4 +- .../q63.native_iceberg_compat/extended.txt | 2 +- .../q64.native_iceberg_compat/extended.txt | 4 +- .../q65.native_iceberg_compat/extended.txt | 4 +- .../q66.native_iceberg_compat/extended.txt | 4 +- .../q67.native_iceberg_compat/extended.txt | 2 +- .../q68.native_iceberg_compat/extended.txt | 2 +- .../q69.native_iceberg_compat/extended.txt | 6 +- .../q7.native_iceberg_compat/extended.txt | 2 +- .../q70.native_iceberg_compat/extended.txt | 4 +- .../q71.native_iceberg_compat/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 2 +- .../q73.native_iceberg_compat/extended.txt | 2 +- .../q74.native_iceberg_compat/extended.txt | 8 +- .../q75.native_iceberg_compat/extended.txt | 12 +- .../q77.native_iceberg_compat/extended.txt | 12 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q79.native_iceberg_compat/extended.txt | 2 +- .../q8.native_iceberg_compat/extended.txt | 2 +- .../q80.native_iceberg_compat/extended.txt | 6 +- .../q81.native_iceberg_compat/extended.txt | 4 +- .../q82.native_iceberg_compat/extended.txt | 2 +- .../extended.txt | 6 +- .../q85.native_iceberg_compat/extended.txt | 2 +- .../q86.native_iceberg_compat/extended.txt | 2 +- .../q87.native_iceberg_compat/extended.txt | 6 +- .../q89.native_iceberg_compat/extended.txt | 2 +- .../q91.native_iceberg_compat/extended.txt | 2 +- .../q92.native_iceberg_compat/extended.txt | 4 +- .../q97.native_iceberg_compat/extended.txt | 4 +- .../q98.native_iceberg_compat/extended.txt | 2 +- .../q1.native_iceberg_compat/extended.txt | 4 +- .../q10.native_iceberg_compat/extended.txt | 6 +- .../q11.native_iceberg_compat/extended.txt | 8 +- .../q12.native_iceberg_compat/extended.txt | 2 +- .../q13.native_iceberg_compat/extended.txt | 2 +- .../q14a.native_iceberg_compat/extended.txt | 48 ++-- .../q14b.native_iceberg_compat/extended.txt | 34 +-- .../q15.native_iceberg_compat/extended.txt | 2 +- .../q17.native_iceberg_compat/extended.txt | 6 +- .../q18.native_iceberg_compat/extended.txt | 2 +- .../q20.native_iceberg_compat/extended.txt | 2 +- .../q21.native_iceberg_compat/extended.txt | 2 +- .../q22.native_iceberg_compat/extended.txt | 2 +- .../q23a.native_iceberg_compat/extended.txt | 10 +- .../q23b.native_iceberg_compat/extended.txt | 10 +- .../q25.native_iceberg_compat/extended.txt | 6 +- .../q26.native_iceberg_compat/extended.txt | 2 +- .../q27.native_iceberg_compat/extended.txt | 2 +- .../q29.native_iceberg_compat/extended.txt | 6 +- .../q30.native_iceberg_compat/extended.txt | 4 +- .../q31.native_iceberg_compat/extended.txt | 12 +- .../q32.native_iceberg_compat/extended.txt | 4 +- .../q33.native_iceberg_compat/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 2 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q36.native_iceberg_compat/extended.txt | 2 +- .../q37.native_iceberg_compat/extended.txt | 2 +- .../q38.native_iceberg_compat/extended.txt | 6 +- .../q39a.native_iceberg_compat/extended.txt | 4 +- .../q39b.native_iceberg_compat/extended.txt | 4 +- .../q4.native_iceberg_compat/extended.txt | 12 +- .../q40.native_iceberg_compat/extended.txt | 2 +- .../q45.native_iceberg_compat/extended.txt | 2 +- .../q46.native_iceberg_compat/extended.txt | 2 +- .../q47.native_iceberg_compat/extended.txt | 6 +- .../q48.native_iceberg_compat/extended.txt | 2 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q5.native_iceberg_compat/extended.txt | 12 +- .../q50.native_iceberg_compat/extended.txt | 2 +- .../q51.native_iceberg_compat/extended.txt | 4 +- .../q53.native_iceberg_compat/extended.txt | 2 +- .../q54.native_iceberg_compat/extended.txt | 6 +- .../q56.native_iceberg_compat/extended.txt | 6 +- .../q57.native_iceberg_compat/extended.txt | 6 +- .../q58.native_iceberg_compat/extended.txt | 6 +- .../q6.native_iceberg_compat/extended.txt | 2 +- .../q60.native_iceberg_compat/extended.txt | 6 +- .../q61.native_iceberg_compat/extended.txt | 4 +- .../q63.native_iceberg_compat/extended.txt | 2 +- .../q64.native_iceberg_compat/extended.txt | 4 +- .../q65.native_iceberg_compat/extended.txt | 4 +- .../q66.native_iceberg_compat/extended.txt | 4 +- .../q67.native_iceberg_compat/extended.txt | 2 +- .../q68.native_iceberg_compat/extended.txt | 2 +- .../q69.native_iceberg_compat/extended.txt | 6 +- .../q7.native_iceberg_compat/extended.txt | 2 +- .../q70.native_iceberg_compat/extended.txt | 4 +- .../q71.native_iceberg_compat/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 2 +- .../q73.native_iceberg_compat/extended.txt | 2 +- .../q74.native_iceberg_compat/extended.txt | 8 +- .../q75.native_iceberg_compat/extended.txt | 12 +- .../q77.native_iceberg_compat/extended.txt | 12 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q79.native_iceberg_compat/extended.txt | 2 +- .../q8.native_iceberg_compat/extended.txt | 2 +- .../q80.native_iceberg_compat/extended.txt | 6 +- .../q81.native_iceberg_compat/extended.txt | 4 +- .../q82.native_iceberg_compat/extended.txt | 2 +- .../q83.native_iceberg_compat/extended.txt | 6 +- .../q85.native_iceberg_compat/extended.txt | 2 +- .../q86.native_iceberg_compat/extended.txt | 2 +- .../q87.native_iceberg_compat/extended.txt | 6 +- .../q89.native_iceberg_compat/extended.txt | 2 +- .../q91.native_iceberg_compat/extended.txt | 2 +- .../q92.native_iceberg_compat/extended.txt | 4 +- .../q97.native_iceberg_compat/extended.txt | 4 +- .../q98.native_iceberg_compat/extended.txt | 2 +- .../q10a.native_iceberg_compat/extended.txt | 6 +- .../q11.native_iceberg_compat/extended.txt | 8 +- .../q12.native_iceberg_compat/extended.txt | 2 +- .../q14.native_iceberg_compat/extended.txt | 34 +-- .../q14a.native_iceberg_compat/extended.txt | 240 +++++++++--------- .../q18a.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/extended.txt | 2 +- .../q22.native_iceberg_compat/extended.txt | 2 +- .../q22a.native_iceberg_compat/extended.txt | 10 +- .../q27a.native_iceberg_compat/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 2 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q35a.native_iceberg_compat/extended.txt | 6 +- .../q36a.native_iceberg_compat/extended.txt | 6 +- .../q47.native_iceberg_compat/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q51a.native_iceberg_compat/extended.txt | 16 +- .../q57.native_iceberg_compat/extended.txt | 6 +- .../q5a.native_iceberg_compat/extended.txt | 36 +-- .../q6.native_iceberg_compat/extended.txt | 2 +- .../q64.native_iceberg_compat/extended.txt | 4 +- .../q67a.native_iceberg_compat/extended.txt | 18 +- .../q70a.native_iceberg_compat/extended.txt | 12 +- .../q72.native_iceberg_compat/extended.txt | 2 +- .../q74.native_iceberg_compat/extended.txt | 8 +- .../q75.native_iceberg_compat/extended.txt | 12 +- .../q77a.native_iceberg_compat/extended.txt | 36 +-- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q80a.native_iceberg_compat/extended.txt | 18 +- .../q86a.native_iceberg_compat/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 2 +- .../q10a.native_iceberg_compat/extended.txt | 6 +- .../q11.native_iceberg_compat/extended.txt | 8 +- .../q12.native_iceberg_compat/extended.txt | 2 +- .../q14.native_iceberg_compat/extended.txt | 34 +-- .../q14a.native_iceberg_compat/extended.txt | 240 +++++++++--------- .../q18a.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/extended.txt | 2 +- .../q22.native_iceberg_compat/extended.txt | 2 +- .../q22a.native_iceberg_compat/extended.txt | 10 +- .../q27a.native_iceberg_compat/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 2 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q35a.native_iceberg_compat/extended.txt | 6 +- .../q36a.native_iceberg_compat/extended.txt | 6 +- .../q47.native_iceberg_compat/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q51a.native_iceberg_compat/extended.txt | 16 +- .../q57.native_iceberg_compat/extended.txt | 6 +- .../q5a.native_iceberg_compat/extended.txt | 36 +-- .../q6.native_iceberg_compat/extended.txt | 2 +- .../q64.native_iceberg_compat/extended.txt | 4 +- .../q67a.native_iceberg_compat/extended.txt | 18 +- .../q70a.native_iceberg_compat/extended.txt | 12 +- .../q72.native_iceberg_compat/extended.txt | 2 +- .../q74.native_iceberg_compat/extended.txt | 8 +- .../q75.native_iceberg_compat/extended.txt | 12 +- .../q77a.native_iceberg_compat/extended.txt | 36 +-- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q80a.native_iceberg_compat/extended.txt | 18 +- .../q86a.native_iceberg_compat/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 2 +- .../q10a.native_iceberg_compat/extended.txt | 6 +- .../q11.native_iceberg_compat/extended.txt | 8 +- .../q12.native_iceberg_compat/extended.txt | 2 +- .../q14.native_iceberg_compat/extended.txt | 34 +-- .../q14a.native_iceberg_compat/extended.txt | 240 +++++++++--------- .../q18a.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/extended.txt | 2 +- .../q22.native_iceberg_compat/extended.txt | 2 +- .../q22a.native_iceberg_compat/extended.txt | 10 +- .../q27a.native_iceberg_compat/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 2 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q35a.native_iceberg_compat/extended.txt | 6 +- .../q36a.native_iceberg_compat/extended.txt | 6 +- .../q47.native_iceberg_compat/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q51a.native_iceberg_compat/extended.txt | 16 +- .../q57.native_iceberg_compat/extended.txt | 6 +- .../q5a.native_iceberg_compat/extended.txt | 36 +-- .../q6.native_iceberg_compat/extended.txt | 2 +- .../q64.native_iceberg_compat/extended.txt | 4 +- .../q67a.native_iceberg_compat/extended.txt | 18 +- .../q70a.native_iceberg_compat/extended.txt | 12 +- .../q72.native_iceberg_compat/extended.txt | 2 +- .../q74.native_iceberg_compat/extended.txt | 8 +- .../q75.native_iceberg_compat/extended.txt | 12 +- .../q77a.native_iceberg_compat/extended.txt | 36 +-- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q80a.native_iceberg_compat/extended.txt | 18 +- .../q86a.native_iceberg_compat/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 2 +- 327 files changed, 1434 insertions(+), 1434 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt index 18e33d8acc..fb562f1904 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -40,7 +40,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt index 1755c11325..28aac3cb2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -47,7 +47,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -74,7 +74,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -96,7 +96,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt index 9c704a0ed6..9d5b4e7b02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt index 2ac8a365e3..95764f1c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ HashAggregate : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt index 9525bd6927..0a57c2d1dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -26,7 +26,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -36,7 +36,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -54,7 +54,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -79,7 +79,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -98,7 +98,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -121,7 +121,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -155,7 +155,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -174,7 +174,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -197,7 +197,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -227,7 +227,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -247,7 +247,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -266,7 +266,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -289,7 +289,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -323,7 +323,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -342,7 +342,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -365,7 +365,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -415,7 +415,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -434,7 +434,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -457,7 +457,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -491,7 +491,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -510,7 +510,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -533,7 +533,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt index 2b714db2ca..8af2e7638a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -20,7 +20,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -97,7 +97,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -120,7 +120,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -154,7 +154,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -173,7 +173,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -196,7 +196,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -231,7 +231,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -261,7 +261,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -280,7 +280,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -303,7 +303,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -337,7 +337,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -356,7 +356,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -379,7 +379,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt index d87e8023b8..4146bedb1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt index a45665ebb8..d7f321c587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt index 907dceaab6..e39c438002 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt index e768681167..8cf12e702d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt index fddcd19496..acbb025ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt index 7901019f25..21c4d589cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt index 840aa90904..4df32e5422 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -66,7 +66,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -121,7 +121,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt index 1cf3a75f81..22c017a703 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -136,7 +136,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -151,7 +151,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt index a45665ebb8..d7f321c587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt index 88c83f0c72..0be67a7720 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt index 3c0aa60951..b587ca369f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt index 6d91a6198e..40c163c75c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt index 164bb96ead..d3dca72ef8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt index ee16bf1f4d..149e1e531e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -94,7 +94,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -115,7 +115,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -136,7 +136,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt index 8f187b8e66..06178b8214 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt index 423f4c671b..07f1d47704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -54,7 +54,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -87,7 +87,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt index acaca69a96..6abb66ed9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt index 64e799aa8e..9e7faaf1bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt index bf503d1bfa..a9b7240a32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -41,7 +41,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt index 80c6af0c0d..cc5311b498 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt index 80c6af0c0d..cc5311b498 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt index 54522ad631..f7d5beaa77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -50,7 +50,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -77,7 +77,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -99,7 +99,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -122,7 +122,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -144,7 +144,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt index 9c1c81dcdf..3a2fb76170 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt index 241c4e86f0..31d2a2406d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt index adadb2dc4b..b9b98bcf7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt index a3eb870b39..8eb85dfa14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt index c940db4cf1..9e232766f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ HashAggregate : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt index b10cf058e0..a3c176a9b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -60,7 +60,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometColumnarToRow : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt index ffeff7c4fd..a2ad1c3fa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -27,7 +27,7 @@ TakeOrderedAndProject : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -51,12 +51,12 @@ TakeOrderedAndProject : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -80,13 +80,13 @@ TakeOrderedAndProject : : :- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- Project : : +- BroadcastHashJoin : : :- BroadcastExchange : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt index c90a5e8c49..10ba922624 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt index 2936547d40..6e7e156e7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt index e2f1062058..27ad1743da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt index 0bf0e75168..9ba26a58f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : : :- Project : : : : : : : : +- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange : : : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : : : +- Project : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt index 1ce83c5a0e..3682583e54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt index 5234ab1b35..1405dc9cbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt index e310b0bf2a..f6a3e56d6b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -63,7 +63,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -97,7 +97,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt index 21ba0466c3..abccd11311 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt index 1ce83c5a0e..3682583e54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt index a29873fa6e..a58b8a8842 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ Project : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -71,7 +71,7 @@ Project : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt index e2f1062058..27ad1743da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt index da58064931..ff3f54fc27 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt @@ -46,7 +46,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt index 9a5a06cf91..b6a5955810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt index d77f6eab96..0592eff15c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -57,7 +57,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt index 2a0632a3bb..ff50de5a95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt index adadb2dc4b..b9b98bcf7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt index 63b1420137..57d98184f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt index 59da61ba7a..180543d784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt index 84bf0cc584..ed290f4f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt index 673dbd4ada..f908086f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -46,7 +46,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt index c2058aa3a3..15f59cd399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : : : +- BroadcastHashJoin : : : : : : : : : :- Filter : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt index 12b5755eb7..1c2b08ba4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt index 7afcaebec8..63679c655c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -52,7 +52,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -144,7 +144,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt index 5ba92ec55f..e8260ee91b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -79,7 +79,7 @@ TakeOrderedAndProject : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -98,7 +98,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -120,7 +120,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt index eec4fb502f..996b4bed53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometColumnarExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange @@ -69,7 +69,7 @@ TakeOrderedAndProject : : +- CometColumnarExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometSort : +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt index 235c1e92f2..9d3f8aea2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt index a240608c19..b33967c1aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt index fccbf0a389..df0da2d681 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -67,7 +67,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt index e6286a9886..4b0542e8fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt index 4da4d56042..bb9078306f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt index ff077f9dd8..96ed4c4678 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -59,7 +59,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -91,7 +91,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt index 3797565aed..59b2bac407 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : : : :- BroadcastExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt index e899c49c4e..bd28ea016b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt index e2b06cf20d..023435c9e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ HashAggregate : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -43,7 +43,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -66,7 +66,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt index e2f1062058..27ad1743da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt index c0b4c3085d..39a53e3d01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt index 874eb62d3c..5b4b326eb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt index 82e4b4b23c..49c3b29af3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt @@ -11,7 +11,7 @@ CometColumnarToRow : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -30,7 +30,7 @@ CometColumnarToRow +- Project +- BroadcastHashJoin :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt index e4611ec552..0a3a70f3f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 18e33d8acc..fb562f1904 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -40,7 +40,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 1755c11325..28aac3cb2c 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 @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -47,7 +47,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -74,7 +74,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -96,7 +96,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow 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 9c704a0ed6..9d5b4e7b02 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 2ac8a365e3..95764f1c1f 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 @@ -14,7 +14,7 @@ HashAggregate : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt index 9525bd6927..0a57c2d1dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -26,7 +26,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -36,7 +36,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -54,7 +54,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -79,7 +79,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -98,7 +98,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -121,7 +121,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -155,7 +155,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -174,7 +174,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -197,7 +197,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -227,7 +227,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -247,7 +247,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -266,7 +266,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -289,7 +289,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -323,7 +323,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -342,7 +342,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -365,7 +365,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -415,7 +415,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -434,7 +434,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -457,7 +457,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -491,7 +491,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -510,7 +510,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -533,7 +533,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt index a02b0fda3c..d6d0b2eba6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -20,7 +20,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -79,7 +79,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -98,7 +98,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -121,7 +121,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -155,7 +155,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -174,7 +174,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -197,7 +197,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -233,7 +233,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -264,7 +264,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -283,7 +283,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -306,7 +306,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -340,7 +340,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -359,7 +359,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -382,7 +382,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 d87e8023b8..4146bedb1e 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 @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt index a45665ebb8..d7f321c587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt index 907dceaab6..e39c438002 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow 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 e768681167..8cf12e702d 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt index fddcd19496..acbb025ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt index 7901019f25..21c4d589cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt index 840aa90904..4df32e5422 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -66,7 +66,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -121,7 +121,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt index 1cf3a75f81..22c017a703 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -136,7 +136,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -151,7 +151,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow 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 a45665ebb8..d7f321c587 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 @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt index 88c83f0c72..0be67a7720 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt index 3c0aa60951..b587ca369f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt index 6d91a6198e..40c163c75c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow 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 164bb96ead..d3dca72ef8 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 @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow 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 ee16bf1f4d..149e1e531e 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 @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -94,7 +94,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -115,7 +115,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -136,7 +136,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow 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 8f187b8e66..06178b8214 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 @@ -10,7 +10,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow 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 423f4c671b..07f1d47704 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 @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -54,7 +54,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -87,7 +87,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 acaca69a96..6abb66ed9d 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 @@ -17,7 +17,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt index 64e799aa8e..9e7faaf1bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt index bf503d1bfa..a9b7240a32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -41,7 +41,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt index 80c6af0c0d..cc5311b498 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt index 80c6af0c0d..cc5311b498 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt index 54522ad631..f7d5beaa77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -50,7 +50,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -77,7 +77,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -99,7 +99,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -122,7 +122,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -144,7 +144,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt index 9c1c81dcdf..3a2fb76170 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt index 241c4e86f0..31d2a2406d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow 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 adadb2dc4b..b9b98bcf7c 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 @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow 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 a3eb870b39..8eb85dfa14 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 @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt index c940db4cf1..9e232766f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ HashAggregate : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt index b10cf058e0..a3c176a9b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -60,7 +60,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometColumnarToRow : +- CometProject 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 ffeff7c4fd..a2ad1c3fa0 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 @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -27,7 +27,7 @@ TakeOrderedAndProject : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -51,12 +51,12 @@ TakeOrderedAndProject : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -80,13 +80,13 @@ TakeOrderedAndProject : : :- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- Project : : +- BroadcastHashJoin : : :- BroadcastExchange : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt index c90a5e8c49..10ba922624 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow 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 2936547d40..6e7e156e7f 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 @@ -21,7 +21,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow 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 e2f1062058..27ad1743da 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 @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 6ecaf6925b..924a4e9ae6 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 @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : : :- Project : : : : : : : : +- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange : : : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : : : +- Project : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow 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 1ce83c5a0e..3682583e54 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 @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 5234ab1b35..1405dc9cbd 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 @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 83febdbd98..49b87191ca 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -100,7 +100,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt index 2191f4ba49..8f75f8eade 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 1ce83c5a0e..3682583e54 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 @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 a29873fa6e..a58b8a8842 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 @@ -18,7 +18,7 @@ Project : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -71,7 +71,7 @@ Project : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow 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 e2f1062058..27ad1743da 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 @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 da58064931..ff3f54fc27 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 @@ -46,7 +46,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : +- CometColumnarToRow 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 9a5a06cf91..b6a5955810 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 @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt index d77f6eab96..0592eff15c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -57,7 +57,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt index 2a0632a3bb..ff50de5a95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow 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 adadb2dc4b..b9b98bcf7c 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 @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt index 63b1420137..57d98184f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt index 59da61ba7a..180543d784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow 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 84bf0cc584..ed290f4f9d 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 @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow 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 673dbd4ada..f908086f9d 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 @@ -19,7 +19,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -46,7 +46,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt index c2058aa3a3..15f59cd399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : : : +- BroadcastHashJoin : : : : : : : : : :- Filter : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow 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 12b5755eb7..1c2b08ba4c 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 @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow 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 7afcaebec8..63679c655c 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 @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -52,7 +52,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -144,7 +144,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 5ba92ec55f..e8260ee91b 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 @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -79,7 +79,7 @@ TakeOrderedAndProject : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -98,7 +98,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -120,7 +120,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow 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 eec4fb502f..996b4bed53 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 @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometColumnarExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange @@ -69,7 +69,7 @@ TakeOrderedAndProject : : +- CometColumnarExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometSort : +- CometExchange 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 235c1e92f2..9d3f8aea2c 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 @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow 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 a240608c19..b33967c1aa 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 @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow 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 fccbf0a389..df0da2d681 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 @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -67,7 +67,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange 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 e6286a9886..4b0542e8fa 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 @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt index 4da4d56042..bb9078306f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt index ff077f9dd8..96ed4c4678 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -59,7 +59,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -91,7 +91,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt index 3797565aed..59b2bac407 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : : : :- BroadcastExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow 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 e899c49c4e..bd28ea016b 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 @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt index e2b06cf20d..023435c9e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ HashAggregate : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -43,7 +43,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -66,7 +66,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow 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 e2f1062058..27ad1743da 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 @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 c0b4c3085d..39a53e3d01 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 @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow 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 874eb62d3c..5b4b326eb9 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 @@ -10,7 +10,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt index 82e4b4b23c..49c3b29af3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt @@ -11,7 +11,7 @@ CometColumnarToRow : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -30,7 +30,7 @@ CometColumnarToRow +- Project +- BroadcastHashJoin :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow 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 e4611ec552..0a3a70f3f9 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 @@ -17,7 +17,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt index 18e33d8acc..fb562f1904 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -40,7 +40,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt index 1755c11325..28aac3cb2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -47,7 +47,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -74,7 +74,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -96,7 +96,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt index 9c704a0ed6..9d5b4e7b02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt index 2ac8a365e3..95764f1c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ HashAggregate : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt index 9525bd6927..0a57c2d1dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -26,7 +26,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -36,7 +36,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -54,7 +54,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -79,7 +79,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -98,7 +98,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -121,7 +121,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -155,7 +155,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -174,7 +174,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -197,7 +197,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -227,7 +227,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -247,7 +247,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -266,7 +266,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -289,7 +289,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -323,7 +323,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -342,7 +342,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -365,7 +365,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -415,7 +415,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -434,7 +434,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -457,7 +457,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -491,7 +491,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -510,7 +510,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -533,7 +533,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt index 2b714db2ca..8af2e7638a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -20,7 +20,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -97,7 +97,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -120,7 +120,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -154,7 +154,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -173,7 +173,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -196,7 +196,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -231,7 +231,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -261,7 +261,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -280,7 +280,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -303,7 +303,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -337,7 +337,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -356,7 +356,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -379,7 +379,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt index d87e8023b8..4146bedb1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt index a45665ebb8..d7f321c587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt index 907dceaab6..e39c438002 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt index e768681167..8cf12e702d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt index fddcd19496..acbb025ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt index 7901019f25..21c4d589cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt index 840aa90904..4df32e5422 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -66,7 +66,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -121,7 +121,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt index 1cf3a75f81..22c017a703 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -136,7 +136,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -151,7 +151,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt index a45665ebb8..d7f321c587 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt index 88c83f0c72..0be67a7720 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt index 3c0aa60951..b587ca369f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt index 6d91a6198e..40c163c75c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt index 164bb96ead..d3dca72ef8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt index ee16bf1f4d..149e1e531e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -94,7 +94,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -115,7 +115,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -136,7 +136,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt index 8f187b8e66..06178b8214 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt index 423f4c671b..07f1d47704 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -54,7 +54,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -87,7 +87,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt index acaca69a96..6abb66ed9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt index 64e799aa8e..9e7faaf1bd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt index bf503d1bfa..a9b7240a32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -41,7 +41,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt index 80c6af0c0d..cc5311b498 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt index 80c6af0c0d..cc5311b498 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt index 54522ad631..f7d5beaa77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -50,7 +50,7 @@ TakeOrderedAndProject : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -77,7 +77,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -99,7 +99,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -122,7 +122,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -144,7 +144,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt index 9c1c81dcdf..3a2fb76170 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ CometColumnarToRow : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt index 241c4e86f0..31d2a2406d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt index adadb2dc4b..b9b98bcf7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt index a3eb870b39..8eb85dfa14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt index c940db4cf1..9e232766f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ HashAggregate : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt index b10cf058e0..a3c176a9b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -60,7 +60,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometColumnarToRow : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt index ffeff7c4fd..a2ad1c3fa0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -27,7 +27,7 @@ TakeOrderedAndProject : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -51,12 +51,12 @@ TakeOrderedAndProject : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -80,13 +80,13 @@ TakeOrderedAndProject : : :- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- Project : : +- BroadcastHashJoin : : :- BroadcastExchange : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt index c90a5e8c49..10ba922624 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt index 2936547d40..6e7e156e7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt index e2f1062058..27ad1743da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt index 0bf0e75168..9ba26a58f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt @@ -29,7 +29,7 @@ TakeOrderedAndProject : : : : : : : :- Project : : : : : : : : +- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- SubqueryBroadcast : : : : : : : : +- BroadcastExchange : : : : : : : : +- CometColumnarToRow @@ -39,7 +39,7 @@ TakeOrderedAndProject : : : : : : : +- Project : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt index 1ce83c5a0e..3682583e54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt index 5234ab1b35..1405dc9cbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt index fb001bbd5c..43a73c6636 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -63,7 +63,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -104,7 +104,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt index 21ba0466c3..abccd11311 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt index 1ce83c5a0e..3682583e54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt index a29873fa6e..a58b8a8842 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ Project : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -71,7 +71,7 @@ Project : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt index e2f1062058..27ad1743da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt index da58064931..ff3f54fc27 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt @@ -46,7 +46,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt index 9a5a06cf91..b6a5955810 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt index d77f6eab96..0592eff15c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -57,7 +57,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt index 2ff013c178..9985a95a0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt index adadb2dc4b..b9b98bcf7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt index 63b1420137..57d98184f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt index 59da61ba7a..180543d784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt index 791ae2720b..586f97186a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt index 673dbd4ada..f908086f9d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -46,7 +46,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt index c2058aa3a3..15f59cd399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : : : +- BroadcastHashJoin : : : : : : : : : :- Filter : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt index 12b5755eb7..1c2b08ba4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt index 7afcaebec8..63679c655c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -52,7 +52,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -144,7 +144,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt index 5ba92ec55f..e8260ee91b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -79,7 +79,7 @@ TakeOrderedAndProject : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -98,7 +98,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -120,7 +120,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt index eec4fb502f..996b4bed53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometColumnarExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange @@ -69,7 +69,7 @@ TakeOrderedAndProject : : +- CometColumnarExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometSort : +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt index 235c1e92f2..9d3f8aea2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt index a240608c19..b33967c1aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt @@ -11,7 +11,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt index fccbf0a389..df0da2d681 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -67,7 +67,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -107,7 +107,7 @@ CometColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt index e6286a9886..4b0542e8fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt index 4da4d56042..bb9078306f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt index 3056740a56..da2d4ae94d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -59,7 +59,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -105,7 +105,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt index 3797565aed..59b2bac407 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : : : :- BroadcastExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt index e899c49c4e..bd28ea016b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt index e2b06cf20d..023435c9e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt @@ -15,7 +15,7 @@ HashAggregate : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -43,7 +43,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -66,7 +66,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt index e2f1062058..27ad1743da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt index c0b4c3085d..39a53e3d01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : : : +- BroadcastExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt index 874eb62d3c..5b4b326eb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ HashAggregate : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -32,7 +32,7 @@ HashAggregate : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt index 82e4b4b23c..49c3b29af3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt @@ -11,7 +11,7 @@ CometColumnarToRow : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -30,7 +30,7 @@ CometColumnarToRow +- Project +- BroadcastHashJoin :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt index e4611ec552..0a3a70f3f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt index f450ef0587..bdeced7e55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt index 12b5755eb7..1c2b08ba4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt index 9c704a0ed6..9d5b4e7b02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt index 2b714db2ca..8af2e7638a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -20,7 +20,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -97,7 +97,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -120,7 +120,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -154,7 +154,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -173,7 +173,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -196,7 +196,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -231,7 +231,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -261,7 +261,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -280,7 +280,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -303,7 +303,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -337,7 +337,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -356,7 +356,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -379,7 +379,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt index 9539604b53..ccbb4fe9da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -62,7 +62,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -87,7 +87,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -129,7 +129,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -163,7 +163,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -182,7 +182,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -205,7 +205,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -234,7 +234,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -254,7 +254,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -273,7 +273,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -296,7 +296,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -330,7 +330,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -349,7 +349,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -372,7 +372,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -401,7 +401,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -421,7 +421,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -440,7 +440,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -463,7 +463,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -497,7 +497,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -516,7 +516,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -539,7 +539,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -574,7 +574,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -584,7 +584,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -599,7 +599,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -617,7 +617,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -642,7 +642,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -661,7 +661,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -684,7 +684,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -718,7 +718,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -737,7 +737,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -760,7 +760,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -789,7 +789,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -809,7 +809,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -828,7 +828,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -851,7 +851,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -885,7 +885,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -904,7 +904,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -927,7 +927,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -956,7 +956,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -976,7 +976,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -995,7 +995,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1018,7 +1018,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1052,7 +1052,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1071,7 +1071,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1094,7 +1094,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1129,7 +1129,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1139,7 +1139,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1154,7 +1154,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1172,7 +1172,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1197,7 +1197,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1216,7 +1216,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1239,7 +1239,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1273,7 +1273,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1292,7 +1292,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1315,7 +1315,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1344,7 +1344,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -1364,7 +1364,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1383,7 +1383,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1406,7 +1406,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1440,7 +1440,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1459,7 +1459,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1482,7 +1482,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1511,7 +1511,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -1531,7 +1531,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1550,7 +1550,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1573,7 +1573,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1607,7 +1607,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1649,7 +1649,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1684,7 +1684,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1694,7 +1694,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1709,7 +1709,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1727,7 +1727,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1752,7 +1752,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1771,7 +1771,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1794,7 +1794,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1828,7 +1828,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1847,7 +1847,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1870,7 +1870,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1899,7 +1899,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -1919,7 +1919,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1938,7 +1938,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1961,7 +1961,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1995,7 +1995,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2014,7 +2014,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2037,7 +2037,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2066,7 +2066,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -2086,7 +2086,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2105,7 +2105,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2128,7 +2128,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2162,7 +2162,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2181,7 +2181,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2204,7 +2204,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2239,7 +2239,7 @@ CometColumnarToRow : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2249,7 +2249,7 @@ CometColumnarToRow : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2264,7 +2264,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -2282,7 +2282,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2307,7 +2307,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2326,7 +2326,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2349,7 +2349,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2383,7 +2383,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2402,7 +2402,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2425,7 +2425,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2454,7 +2454,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -2474,7 +2474,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2493,7 +2493,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2516,7 +2516,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2550,7 +2550,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2569,7 +2569,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2592,7 +2592,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2621,7 +2621,7 @@ CometColumnarToRow : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -2641,7 +2641,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2660,7 +2660,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2683,7 +2683,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2717,7 +2717,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2736,7 +2736,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2759,7 +2759,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt index 636e8a5322..6909da9aa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -72,7 +72,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -121,7 +121,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -219,7 +219,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt index e768681167..8cf12e702d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt index 05ee5744ab..614af2a9b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt index d3043a5277..702314cced 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -88,7 +88,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -125,7 +125,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -162,7 +162,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt index 0c5aac8292..1378dbd839 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -91,7 +91,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt index f450ef0587..bdeced7e55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt index a0e30dcb5b..fc61a5c7eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -96,7 +96,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt index a3eb870b39..8eb85dfa14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt index b10cf058e0..a3c176a9b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -60,7 +60,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometColumnarToRow : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt index cc2068bae6..ebbf4c78f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -91,7 +91,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -112,7 +112,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -149,7 +149,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -175,7 +175,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -208,7 +208,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -229,7 +229,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt index 5234ab1b35..1405dc9cbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt index 85d0f4d4cb..b028a77978 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -31,7 +31,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -55,12 +55,12 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -84,13 +84,13 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- BroadcastHashJoin : : : :- BroadcastExchange : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- CometColumnarToRow : : : +- CometProject @@ -127,7 +127,7 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -137,7 +137,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -161,12 +161,12 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -190,13 +190,13 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- BroadcastHashJoin : : : :- BroadcastExchange : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- CometColumnarToRow : : : +- CometProject @@ -233,7 +233,7 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -243,7 +243,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -267,12 +267,12 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -296,13 +296,13 @@ CometColumnarToRow : : :- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- Project : : +- BroadcastHashJoin : : :- BroadcastExchange : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt index 21ba0466c3..abccd11311 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt index da58064931..ff3f54fc27 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt @@ -46,7 +46,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt index d40b7fd814..f37b1a2e9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -96,7 +96,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -134,7 +134,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -172,7 +172,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -210,7 +210,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -248,7 +248,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -286,7 +286,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -324,7 +324,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index be77e5eca2..1b160f655d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -52,7 +52,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -78,7 +78,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -112,7 +112,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -138,7 +138,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -172,7 +172,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt index c2058aa3a3..15f59cd399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : : : +- BroadcastHashJoin : : : : : : : : : :- Filter : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt index 12b5755eb7..1c2b08ba4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt index 7afcaebec8..63679c655c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -52,7 +52,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -144,7 +144,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt index 0fef4fef27..54f4396187 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -48,7 +48,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -83,7 +83,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -102,7 +102,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -124,7 +124,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -183,7 +183,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -204,7 +204,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -218,7 +218,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -237,7 +237,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -259,7 +259,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -291,7 +291,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -318,7 +318,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -339,7 +339,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -353,7 +353,7 @@ CometColumnarToRow : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -372,7 +372,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -394,7 +394,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt index eec4fb502f..996b4bed53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometColumnarExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange @@ -69,7 +69,7 @@ TakeOrderedAndProject : : +- CometColumnarExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometSort : +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt index c2f4c7cc7c..86bf4d73b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -70,7 +70,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange @@ -110,7 +110,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -157,7 +157,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -202,7 +202,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange @@ -242,7 +242,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -289,7 +289,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -334,7 +334,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -374,7 +374,7 @@ CometColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt index f7d1e34134..41710f125f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt index e6bdee919f..1894bb8648 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt index f450ef0587..bdeced7e55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 12b5755eb7..1c2b08ba4c 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 @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow 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 9c704a0ed6..9d5b4e7b02 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt index a02b0fda3c..d6d0b2eba6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -20,7 +20,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -79,7 +79,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -98,7 +98,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -121,7 +121,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -155,7 +155,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -174,7 +174,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -197,7 +197,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -233,7 +233,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -264,7 +264,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -283,7 +283,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -306,7 +306,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -340,7 +340,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -359,7 +359,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -382,7 +382,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt index 9539604b53..ccbb4fe9da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -62,7 +62,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -87,7 +87,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -129,7 +129,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -163,7 +163,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -182,7 +182,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -205,7 +205,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -234,7 +234,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -254,7 +254,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -273,7 +273,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -296,7 +296,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -330,7 +330,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -349,7 +349,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -372,7 +372,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -401,7 +401,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -421,7 +421,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -440,7 +440,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -463,7 +463,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -497,7 +497,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -516,7 +516,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -539,7 +539,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -574,7 +574,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -584,7 +584,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -599,7 +599,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -617,7 +617,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -642,7 +642,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -661,7 +661,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -684,7 +684,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -718,7 +718,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -737,7 +737,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -760,7 +760,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -789,7 +789,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -809,7 +809,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -828,7 +828,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -851,7 +851,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -885,7 +885,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -904,7 +904,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -927,7 +927,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -956,7 +956,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -976,7 +976,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -995,7 +995,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1018,7 +1018,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1052,7 +1052,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1071,7 +1071,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1094,7 +1094,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1129,7 +1129,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1139,7 +1139,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1154,7 +1154,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1172,7 +1172,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1197,7 +1197,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1216,7 +1216,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1239,7 +1239,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1273,7 +1273,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1292,7 +1292,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1315,7 +1315,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1344,7 +1344,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -1364,7 +1364,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1383,7 +1383,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1406,7 +1406,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1440,7 +1440,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1459,7 +1459,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1482,7 +1482,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1511,7 +1511,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -1531,7 +1531,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1550,7 +1550,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1573,7 +1573,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1607,7 +1607,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1649,7 +1649,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1684,7 +1684,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1694,7 +1694,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1709,7 +1709,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1727,7 +1727,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1752,7 +1752,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1771,7 +1771,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1794,7 +1794,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1828,7 +1828,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1847,7 +1847,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1870,7 +1870,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1899,7 +1899,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -1919,7 +1919,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1938,7 +1938,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1961,7 +1961,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1995,7 +1995,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2014,7 +2014,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2037,7 +2037,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2066,7 +2066,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -2086,7 +2086,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2105,7 +2105,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2128,7 +2128,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2162,7 +2162,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2181,7 +2181,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2204,7 +2204,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2239,7 +2239,7 @@ CometColumnarToRow : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2249,7 +2249,7 @@ CometColumnarToRow : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2264,7 +2264,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -2282,7 +2282,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2307,7 +2307,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2326,7 +2326,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2349,7 +2349,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2383,7 +2383,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2402,7 +2402,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2425,7 +2425,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2454,7 +2454,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -2474,7 +2474,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2493,7 +2493,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2516,7 +2516,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2550,7 +2550,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2569,7 +2569,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2592,7 +2592,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2621,7 +2621,7 @@ CometColumnarToRow : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -2641,7 +2641,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2660,7 +2660,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2683,7 +2683,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2717,7 +2717,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2736,7 +2736,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2759,7 +2759,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt index 636e8a5322..6909da9aa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -72,7 +72,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -121,7 +121,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -219,7 +219,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow 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 e768681167..8cf12e702d 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 @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt index 05ee5744ab..614af2a9b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt index d3043a5277..702314cced 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -88,7 +88,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -125,7 +125,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -162,7 +162,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt index 0c5aac8292..1378dbd839 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -91,7 +91,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt index f450ef0587..bdeced7e55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 a0e30dcb5b..fc61a5c7eb 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 @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -96,7 +96,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow 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 a3eb870b39..8eb85dfa14 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 @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt index b10cf058e0..a3c176a9b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -60,7 +60,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometColumnarToRow : +- CometProject 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 cc2068bae6..ebbf4c78f7 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 @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -91,7 +91,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -112,7 +112,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -149,7 +149,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -175,7 +175,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -208,7 +208,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -229,7 +229,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow 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 5234ab1b35..1405dc9cbd 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 @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 85d0f4d4cb..b028a77978 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 @@ -21,7 +21,7 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -31,7 +31,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -55,12 +55,12 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -84,13 +84,13 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- BroadcastHashJoin : : : :- BroadcastExchange : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- CometColumnarToRow : : : +- CometProject @@ -127,7 +127,7 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -137,7 +137,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -161,12 +161,12 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -190,13 +190,13 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- BroadcastHashJoin : : : :- BroadcastExchange : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- CometColumnarToRow : : : +- CometProject @@ -233,7 +233,7 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -243,7 +243,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -267,12 +267,12 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -296,13 +296,13 @@ CometColumnarToRow : : :- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- Project : : +- BroadcastHashJoin : : :- BroadcastExchange : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt index 2191f4ba49..8f75f8eade 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 da58064931..ff3f54fc27 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 @@ -46,7 +46,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt index d40b7fd814..f37b1a2e9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -96,7 +96,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -134,7 +134,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -172,7 +172,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -210,7 +210,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -248,7 +248,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -286,7 +286,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -324,7 +324,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow 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 be77e5eca2..1b160f655d 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 @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -52,7 +52,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -78,7 +78,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -112,7 +112,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -138,7 +138,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -172,7 +172,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt index c2058aa3a3..15f59cd399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : : : +- BroadcastHashJoin : : : : : : : : : :- Filter : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : +- CometColumnarToRow 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 12b5755eb7..1c2b08ba4c 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 @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow 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 7afcaebec8..63679c655c 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 @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -52,7 +52,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -144,7 +144,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow 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 0fef4fef27..54f4396187 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 @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -48,7 +48,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -83,7 +83,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -102,7 +102,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -124,7 +124,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -183,7 +183,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -204,7 +204,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -218,7 +218,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -237,7 +237,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -259,7 +259,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -291,7 +291,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -318,7 +318,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -339,7 +339,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -353,7 +353,7 @@ CometColumnarToRow : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -372,7 +372,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -394,7 +394,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow 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 eec4fb502f..996b4bed53 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 @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometColumnarExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange @@ -69,7 +69,7 @@ TakeOrderedAndProject : : +- CometColumnarExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometSort : +- CometExchange 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 c2f4c7cc7c..86bf4d73b8 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 @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -70,7 +70,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange @@ -110,7 +110,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -157,7 +157,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -202,7 +202,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange @@ -242,7 +242,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -289,7 +289,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -334,7 +334,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -374,7 +374,7 @@ CometColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange 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 f7d1e34134..41710f125f 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 @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow 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 e6bdee919f..1894bb8648 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 @@ -16,7 +16,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt index f450ef0587..bdeced7e55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt index 12b5755eb7..1c2b08ba4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt index 9c704a0ed6..9d5b4e7b02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt index 2b714db2ca..8af2e7638a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt @@ -10,7 +10,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -20,7 +20,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -30,7 +30,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -48,7 +48,7 @@ TakeOrderedAndProject : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -97,7 +97,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -120,7 +120,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -154,7 +154,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -173,7 +173,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -196,7 +196,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -231,7 +231,7 @@ TakeOrderedAndProject : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -261,7 +261,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -280,7 +280,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -303,7 +303,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -337,7 +337,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -356,7 +356,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -379,7 +379,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt index 9539604b53..ccbb4fe9da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -29,7 +29,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -62,7 +62,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -87,7 +87,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -129,7 +129,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -163,7 +163,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -182,7 +182,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -205,7 +205,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -234,7 +234,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -254,7 +254,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -273,7 +273,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -296,7 +296,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -330,7 +330,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -349,7 +349,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -372,7 +372,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -401,7 +401,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -421,7 +421,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -440,7 +440,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -463,7 +463,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -497,7 +497,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -516,7 +516,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -539,7 +539,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -574,7 +574,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -584,7 +584,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -599,7 +599,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -617,7 +617,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -642,7 +642,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -661,7 +661,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -684,7 +684,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -718,7 +718,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -737,7 +737,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -760,7 +760,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -789,7 +789,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -809,7 +809,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -828,7 +828,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -851,7 +851,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -885,7 +885,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -904,7 +904,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -927,7 +927,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -956,7 +956,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -976,7 +976,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -995,7 +995,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1018,7 +1018,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1052,7 +1052,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1071,7 +1071,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1094,7 +1094,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1129,7 +1129,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1139,7 +1139,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1154,7 +1154,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1172,7 +1172,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1197,7 +1197,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1216,7 +1216,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1239,7 +1239,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1273,7 +1273,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1292,7 +1292,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1315,7 +1315,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1344,7 +1344,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -1364,7 +1364,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1383,7 +1383,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1406,7 +1406,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1440,7 +1440,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1459,7 +1459,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1482,7 +1482,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1511,7 +1511,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -1531,7 +1531,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1550,7 +1550,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1573,7 +1573,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1607,7 +1607,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1626,7 +1626,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1649,7 +1649,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1684,7 +1684,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1694,7 +1694,7 @@ CometColumnarToRow : : : :- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1709,7 +1709,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -1727,7 +1727,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1752,7 +1752,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1771,7 +1771,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1794,7 +1794,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1828,7 +1828,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1847,7 +1847,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1870,7 +1870,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1899,7 +1899,7 @@ CometColumnarToRow : : : :- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- Project @@ -1919,7 +1919,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1938,7 +1938,7 @@ CometColumnarToRow : : : : : : : +- BroadcastHashJoin : : : : : : : :- Filter : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- ReusedSubquery : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -1961,7 +1961,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1995,7 +1995,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2014,7 +2014,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2037,7 +2037,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2066,7 +2066,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -2086,7 +2086,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2105,7 +2105,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2128,7 +2128,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2162,7 +2162,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2181,7 +2181,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2204,7 +2204,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2239,7 +2239,7 @@ CometColumnarToRow : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2249,7 +2249,7 @@ CometColumnarToRow : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2264,7 +2264,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -2282,7 +2282,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2307,7 +2307,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2326,7 +2326,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2349,7 +2349,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2383,7 +2383,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2402,7 +2402,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2425,7 +2425,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2454,7 +2454,7 @@ CometColumnarToRow : : :- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- Project @@ -2474,7 +2474,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2493,7 +2493,7 @@ CometColumnarToRow : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2516,7 +2516,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2550,7 +2550,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2569,7 +2569,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2592,7 +2592,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2621,7 +2621,7 @@ CometColumnarToRow : :- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- Project @@ -2641,7 +2641,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2660,7 +2660,7 @@ CometColumnarToRow : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2683,7 +2683,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2717,7 +2717,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2736,7 +2736,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2759,7 +2759,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt index 636e8a5322..6909da9aa4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -72,7 +72,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -121,7 +121,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ TakeOrderedAndProject : : : : : : +- BroadcastHashJoin : : : : : : :- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -219,7 +219,7 @@ TakeOrderedAndProject : : : : : +- BroadcastHashJoin : : : : : :- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt index e768681167..8cf12e702d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt index 05ee5744ab..614af2a9b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt @@ -12,7 +12,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt index d3043a5277..702314cced 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -88,7 +88,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -125,7 +125,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -162,7 +162,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt index 0c5aac8292..1378dbd839 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -91,7 +91,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt index 494d98d409..ba55221147 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt index 68b088493c..f1832f777a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt @@ -19,7 +19,7 @@ TakeOrderedAndProject : : : : +- Project : : : : +- BroadcastHashJoin : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -35,7 +35,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt index f450ef0587..bdeced7e55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -34,7 +34,7 @@ TakeOrderedAndProject : : :- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -44,7 +44,7 @@ TakeOrderedAndProject : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt index a0e30dcb5b..fc61a5c7eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -96,7 +96,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt index a3eb870b39..8eb85dfa14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt index b10cf058e0..a3c176a9b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt @@ -24,7 +24,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -60,7 +60,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject @@ -91,7 +91,7 @@ CometColumnarToRow : : +- Project : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometColumnarToRow : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt index cc2068bae6..ebbf4c78f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt @@ -32,7 +32,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -91,7 +91,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -112,7 +112,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -149,7 +149,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -175,7 +175,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -208,7 +208,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -229,7 +229,7 @@ TakeOrderedAndProject +- BroadcastHashJoin :- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt index 5234ab1b35..1405dc9cbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt @@ -28,7 +28,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -102,7 +102,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt index 85d0f4d4cb..b028a77978 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -31,7 +31,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -55,12 +55,12 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -84,13 +84,13 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- BroadcastHashJoin : : : :- BroadcastExchange : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- CometColumnarToRow : : : +- CometProject @@ -127,7 +127,7 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -137,7 +137,7 @@ CometColumnarToRow : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -161,12 +161,12 @@ CometColumnarToRow : : : : :- Project : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -190,13 +190,13 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- BroadcastHashJoin : : : :- BroadcastExchange : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- CometColumnarToRow : : : +- CometProject @@ -233,7 +233,7 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -243,7 +243,7 @@ CometColumnarToRow : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -267,12 +267,12 @@ CometColumnarToRow : : : :- Project : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -296,13 +296,13 @@ CometColumnarToRow : : :- Project : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- Project : : +- BroadcastHashJoin : : :- BroadcastExchange : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometColumnarToRow : : +- CometProject diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt index 21ba0466c3..abccd11311 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt index da58064931..ff3f54fc27 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt @@ -46,7 +46,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -168,7 +168,7 @@ CometColumnarToRow : : : : : : : : : : : : : : : : :- BroadcastExchange : : : : : : : : : : : : : : : : : +- Filter : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt index ab9cfad7ba..fdea4192ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -55,7 +55,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -93,7 +93,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -131,7 +131,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -169,7 +169,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -207,7 +207,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -245,7 +245,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -283,7 +283,7 @@ TakeOrderedAndProject : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -321,7 +321,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt index b8c9813039..dec03f15f7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -51,7 +51,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -77,7 +77,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -110,7 +110,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -136,7 +136,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -169,7 +169,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt index c2058aa3a3..15f59cd399 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt @@ -27,7 +27,7 @@ CometColumnarToRow : : : : : : : : : +- BroadcastHashJoin : : : : : : : : : :- Filter : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast : : : : : : : : : : +- BroadcastExchange : : : : : : : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt index 12b5755eb7..1c2b08ba4c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt @@ -20,7 +20,7 @@ TakeOrderedAndProject : : : : +- BroadcastExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -46,7 +46,7 @@ TakeOrderedAndProject : : : +- BroadcastExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -73,7 +73,7 @@ TakeOrderedAndProject : : +- BroadcastExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -95,7 +95,7 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Filter : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt index 7afcaebec8..63679c655c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -52,7 +52,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -78,7 +78,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -144,7 +144,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -170,7 +170,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt index 0fef4fef27..54f4396187 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt @@ -21,7 +21,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -48,7 +48,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -69,7 +69,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -83,7 +83,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -102,7 +102,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -124,7 +124,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : : : : +- BroadcastHashJoin : : : : :- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -183,7 +183,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -204,7 +204,7 @@ CometColumnarToRow : : : +- Project : : : +- BroadcastHashJoin : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -218,7 +218,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -237,7 +237,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- ReusedSubquery : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -259,7 +259,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -291,7 +291,7 @@ CometColumnarToRow : : : +- BroadcastHashJoin : : : :- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -318,7 +318,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -339,7 +339,7 @@ CometColumnarToRow : : +- Project : : +- BroadcastHashJoin : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -353,7 +353,7 @@ CometColumnarToRow : +- Project : +- BroadcastHashJoin : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow @@ -372,7 +372,7 @@ CometColumnarToRow : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- BroadcastExchange : : +- CometColumnarToRow @@ -394,7 +394,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt index eec4fb502f..996b4bed53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject : : : : +- CometColumnarExchange : : : : +- Filter : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -45,7 +45,7 @@ TakeOrderedAndProject : : : +- CometColumnarExchange : : : +- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange @@ -69,7 +69,7 @@ TakeOrderedAndProject : : +- CometColumnarExchange : : +- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- ReusedSubquery : +- CometSort : +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt index c2f4c7cc7c..86bf4d73b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt @@ -25,7 +25,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -70,7 +70,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange @@ -110,7 +110,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -157,7 +157,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast : : : : : : : +- BroadcastExchange : : : : : : : +- CometColumnarToRow @@ -202,7 +202,7 @@ CometColumnarToRow : : : : : : : +- CometColumnarExchange : : : : : : : +- Filter : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange @@ -242,7 +242,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -289,7 +289,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -334,7 +334,7 @@ CometColumnarToRow : : : : : : +- CometColumnarExchange : : : : : : +- Filter : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange @@ -374,7 +374,7 @@ CometColumnarToRow : : : : : +- CometColumnarExchange : : : : : +- Filter : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt index f7d1e34134..41710f125f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt @@ -18,7 +18,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -49,7 +49,7 @@ TakeOrderedAndProject : : +- BroadcastHashJoin : : :- Filter : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt index e6bdee919f..1894bb8648 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt @@ -16,7 +16,7 @@ CometColumnarToRow : +- BroadcastHashJoin : :- Filter : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported] + : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow From a8049202f5592eb4d947d212047a6da9fdefdc73 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 15:07:16 -0500 Subject: [PATCH 11/38] Add test to reproduce Unexpected subquery plan type: org.apache.spark.sql.execution.SubqueryExec --- .../apache/comet/exec/CometExecSuite.scala | 65 +++++++++++++++++++ 1 file changed, 65 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index daf90cd884..76675a8201 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -281,6 +281,71 @@ class CometExecSuite extends CometTestBase { } } + test("DPP with native_datafusion scan - SubqueryExec (non-broadcast DPP)") { + // This test triggers SubqueryExec instead of SubqueryBroadcastExec. + // Reproduces the failure in Spark's DynamicPartitionPruningV1SuiteAEOff. + // Setup matches DynamicPartitionPruningSuiteBase.beforeAll exactly. + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, + // Must disable AQE (like DynamicPartitionPruningV1SuiteAEOff) + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true") { + withTable("fact_sk", "dim_store") { + // Exact data from DynamicPartitionPruningSuiteBase + val factData = Seq( + (1000, 1, 1, 10), (1010, 2, 1, 10), (1020, 2, 1, 10), (1030, 3, 2, 10), + (1040, 3, 2, 50), (1050, 3, 2, 50), (1060, 3, 2, 50), (1070, 4, 2, 10), + (1080, 4, 3, 20), (1090, 4, 3, 10), (1100, 4, 3, 10), (1110, 5, 3, 10), + (1120, 6, 4, 10), (1130, 7, 4, 50), (1140, 8, 4, 50), (1150, 9, 1, 20), + (1160, 10, 1, 20), (1170, 11, 1, 30), (1180, 12, 2, 20), (1190, 13, 2, 20), + (1200, 14, 3, 40), (1200, 15, 3, 70), (1210, 16, 4, 10), (1220, 17, 4, 20), + (1230, 18, 4, 20), (1240, 19, 5, 40), (1250, 20, 5, 40), (1260, 21, 5, 40), + (1270, 22, 5, 50), (1280, 23, 1, 50), (1290, 24, 1, 50), (1300, 25, 1, 50)) + spark + .createDataFrame(factData) + .toDF("date_id", "store_id", "product_id", "units_sold") + .write + .partitionBy("store_id") + .format("parquet") + .saveAsTable("fact_sk") + + val storeData = Seq( + (1, "North-Holland", "NL"), (2, "South-Holland", "NL"), (3, "Bavaria", "DE"), + (4, "California", "US"), (5, "Texas", "US"), (6, "Texas", "US")) + spark + .createDataFrame(storeData) + .toDF("store_id", "state_province", "country") + .write + .format("parquet") + .saveAsTable("dim_store") + + // Only analyze dim_store (like the Spark test) + sql("ANALYZE TABLE dim_store COMPUTE STATISTICS FOR COLUMNS store_id") + + // Exact test from "simple inner join triggers DPP with mock-up tables" + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { + + val query = + """SELECT f.date_id, f.store_id FROM fact_sk f + |JOIN dim_store s ON f.store_id = s.store_id AND s.country = 'NL'""".stripMargin + + val df = sql(query) + val planStr = df.queryExecution.executedPlan.toString + assert( + planStr.contains("dynamicpruning"), + s"Expected dynamic pruning in plan but got:\n$planStr") + + checkSparkAnswer(df) + } + } + } + } + test("ShuffleQueryStageExec could be direct child node of CometBroadcastExchangeExec") { withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val table = "src" From 4352c0476cefc77bb9c3639348dbe8bec9c6b668 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 15:12:11 -0500 Subject: [PATCH 12/38] Add failing Iceberg test too. --- .../comet/CometIcebergNativeSuite.scala | 73 +++++++++++++++++++ 1 file changed, 73 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 033b634e0f..1861dd2a45 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -25,6 +25,7 @@ import java.nio.file.Files import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.comet.CometIcebergNativeScanExec import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StringType, TimestampType} import org.apache.comet.iceberg.RESTCatalogHelper @@ -2454,6 +2455,78 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { } } + test("runtime filtering - DPP with SubqueryExec (non-broadcast)") { + // This test triggers SubqueryExec instead of SubqueryBroadcastExec. + // Reproduces the failure pattern from Spark's DynamicPartitionPruningV1SuiteAEOff. + assume(icebergAvailable, "Iceberg not available") + withTempIcebergDir { warehouseDir => + val dimDir = new File(warehouseDir, "dim_parquet_subquery") + withSQLConf( + "spark.sql.catalog.subquery_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.subquery_cat.type" -> "hadoop", + "spark.sql.catalog.subquery_cat.warehouse" -> warehouseDir.getAbsolutePath, + // Must disable AQE (like DynamicPartitionPruningV1SuiteAEOff) + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + // Key configs to trigger SubqueryExec instead of SubqueryBroadcastExec + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false", + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // Create partitioned Iceberg table (fact table) + spark.sql(""" + CREATE TABLE subquery_cat.db.fact_table ( + id BIGINT, + data STRING, + date DATE + ) USING iceberg + PARTITIONED BY (date) + """) + + // Insert data across multiple partitions + spark.sql(""" + INSERT INTO subquery_cat.db.fact_table VALUES + (1, 'a', DATE '1970-01-01'), + (2, 'b', DATE '1970-01-02'), + (3, 'c', DATE '1970-01-02'), + (4, 'd', DATE '1970-01-03'), + (5, 'e', DATE '1970-01-01'), + (6, 'f', DATE '1970-01-02'), + (7, 'g', DATE '1970-01-03'), + (8, 'h', DATE '1970-01-01') + """) + + // Create dimension table and analyze it (like Spark's DPP test suite) + spark + .createDataFrame(Seq((1L, java.sql.Date.valueOf("1970-01-02")))) + .toDF("id", "date") + .write + .parquet(dimDir.getAbsolutePath) + spark.read.parquet(dimDir.getAbsolutePath).createOrReplaceTempView("dim_subquery") + spark.catalog.cacheTable("dim_subquery") + sql("ANALYZE TABLE dim_subquery COMPUTE STATISTICS FOR COLUMNS id, date") + + // NO broadcast hint - allows SubqueryExec to be used + val query = + """SELECT f.* FROM subquery_cat.db.fact_table f + |JOIN dim_subquery d ON f.date = d.date AND d.id = 1 + |ORDER BY f.id""".stripMargin + + val df = spark.sql(query) + val planStr = df.queryExecution.executedPlan.toString + assert( + planStr.contains("dynamicpruning"), + s"Expected dynamic pruning in plan but got:\n$planStr") + + checkSparkAnswer(df) + + spark.sql("DROP TABLE subquery_cat.db.fact_table") + } + } + } + // Regression test for a user reported issue test("double partitioning with range filter on top-level partition") { assume(icebergAvailable, "Iceberg not available") From 7f0004ffcda05ef982caac03ece88bfdb014c3dd Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 15:18:37 -0500 Subject: [PATCH 13/38] Handle SubqueryExec in addition to SubqueryBroadcastExec. Add relevant tests. --- .../apache/spark/sql/comet/CometIcebergNativeScanExec.scala | 4 ++-- .../org/apache/spark/sql/comet/CometNativeScanExec.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index d18b2ccf01..e1b488269b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Attribute, DynamicPruningExpression, SortOrder} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} -import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec} +import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec, SubqueryExec} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -117,7 +117,7 @@ case class CometIcebergNativeScanExec( e.plan match { case sab: SubqueryAdaptiveBroadcastExec => resolveSubqueryAdaptiveBroadcast(sab, e) - case _: SubqueryBroadcastExec => + case _: SubqueryBroadcastExec | _: SubqueryExec => e.updateResult() case other => throw new IllegalStateException( diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 8466a9a4eb..1cca904d93 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -105,7 +105,7 @@ case class CometNativeScanExec( e.plan match { case sab: SubqueryAdaptiveBroadcastExec => resolveSubqueryAdaptiveBroadcast(sab, e) - case _: SubqueryBroadcastExec => + case _: SubqueryBroadcastExec | _: SubqueryExec => e.updateResult() case other => throw new IllegalStateException( From 959d517fe13394ffe61f177f574797e293418511 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 15:18:52 -0500 Subject: [PATCH 14/38] Handle SubqueryExec in addition to SubqueryBroadcastExec. Add relevant tests. --- .../apache/comet/exec/CometExecSuite.scala | 48 +++++++++++++++---- 1 file changed, 38 insertions(+), 10 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 76675a8201..a44ff37092 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -295,14 +295,38 @@ class CometExecSuite extends CometTestBase { withTable("fact_sk", "dim_store") { // Exact data from DynamicPartitionPruningSuiteBase val factData = Seq( - (1000, 1, 1, 10), (1010, 2, 1, 10), (1020, 2, 1, 10), (1030, 3, 2, 10), - (1040, 3, 2, 50), (1050, 3, 2, 50), (1060, 3, 2, 50), (1070, 4, 2, 10), - (1080, 4, 3, 20), (1090, 4, 3, 10), (1100, 4, 3, 10), (1110, 5, 3, 10), - (1120, 6, 4, 10), (1130, 7, 4, 50), (1140, 8, 4, 50), (1150, 9, 1, 20), - (1160, 10, 1, 20), (1170, 11, 1, 30), (1180, 12, 2, 20), (1190, 13, 2, 20), - (1200, 14, 3, 40), (1200, 15, 3, 70), (1210, 16, 4, 10), (1220, 17, 4, 20), - (1230, 18, 4, 20), (1240, 19, 5, 40), (1250, 20, 5, 40), (1260, 21, 5, 40), - (1270, 22, 5, 50), (1280, 23, 1, 50), (1290, 24, 1, 50), (1300, 25, 1, 50)) + (1000, 1, 1, 10), + (1010, 2, 1, 10), + (1020, 2, 1, 10), + (1030, 3, 2, 10), + (1040, 3, 2, 50), + (1050, 3, 2, 50), + (1060, 3, 2, 50), + (1070, 4, 2, 10), + (1080, 4, 3, 20), + (1090, 4, 3, 10), + (1100, 4, 3, 10), + (1110, 5, 3, 10), + (1120, 6, 4, 10), + (1130, 7, 4, 50), + (1140, 8, 4, 50), + (1150, 9, 1, 20), + (1160, 10, 1, 20), + (1170, 11, 1, 30), + (1180, 12, 2, 20), + (1190, 13, 2, 20), + (1200, 14, 3, 40), + (1200, 15, 3, 70), + (1210, 16, 4, 10), + (1220, 17, 4, 20), + (1230, 18, 4, 20), + (1240, 19, 5, 40), + (1250, 20, 5, 40), + (1260, 21, 5, 40), + (1270, 22, 5, 50), + (1280, 23, 1, 50), + (1290, 24, 1, 50), + (1300, 25, 1, 50)) spark .createDataFrame(factData) .toDF("date_id", "store_id", "product_id", "units_sold") @@ -312,8 +336,12 @@ class CometExecSuite extends CometTestBase { .saveAsTable("fact_sk") val storeData = Seq( - (1, "North-Holland", "NL"), (2, "South-Holland", "NL"), (3, "Bavaria", "DE"), - (4, "California", "US"), (5, "Texas", "US"), (6, "Texas", "US")) + (1, "North-Holland", "NL"), + (2, "South-Holland", "NL"), + (3, "Bavaria", "DE"), + (4, "California", "US"), + (5, "Texas", "US"), + (6, "Texas", "US")) spark .createDataFrame(storeData) .toDF("store_id", "state_province", "country") From 69b3559af610b5e1305dfcf0b4b6378162d5d301 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 15:54:07 -0500 Subject: [PATCH 15/38] Handle SubqueryExec in addition to SubqueryBroadcastExec. Add relevant tests. --- .../comet/CometIcebergNativeScanExec.scala | 4 +- .../spark/sql/comet/CometNativeScanExec.scala | 2 +- .../comet/CometIcebergNativeSuite.scala | 28 +++++--- .../apache/comet/exec/CometExecSuite.scala | 65 ++++++++++++++++++- 4 files changed, 87 insertions(+), 12 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index e1b488269b..7db84bbfc3 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Attribute, DynamicPruningExpression, SortOrder} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} -import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec, SubqueryExec} +import org.apache.spark.sql.execution.{InSubqueryExec, ReusedSubqueryExec, SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec, SubqueryExec} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -117,7 +117,7 @@ case class CometIcebergNativeScanExec( e.plan match { case sab: SubqueryAdaptiveBroadcastExec => resolveSubqueryAdaptiveBroadcast(sab, e) - case _: SubqueryBroadcastExec | _: SubqueryExec => + case _: SubqueryBroadcastExec | _: SubqueryExec | _: ReusedSubqueryExec => e.updateResult() case other => throw new IllegalStateException( diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 1cca904d93..4f7ceeacc2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -105,7 +105,7 @@ case class CometNativeScanExec( e.plan match { case sab: SubqueryAdaptiveBroadcastExec => resolveSubqueryAdaptiveBroadcast(sab, e) - case _: SubqueryBroadcastExec | _: SubqueryExec => + case _: SubqueryBroadcastExec | _: SubqueryExec | _: ReusedSubqueryExec => e.updateResult() case other => throw new IllegalStateException( diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index 1861dd2a45..dc52a6df75 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -23,8 +23,9 @@ import java.io.File import java.nio.file.Files import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression import org.apache.spark.sql.comet.CometIcebergNativeScanExec -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{InSubqueryExec, SparkPlan, SubqueryExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StringType, TimestampType} @@ -2455,9 +2456,8 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { } } - test("runtime filtering - DPP with SubqueryExec (non-broadcast)") { - // This test triggers SubqueryExec instead of SubqueryBroadcastExec. - // Reproduces the failure pattern from Spark's DynamicPartitionPruningV1SuiteAEOff. + test("runtime filtering - DPP with non-broadcast join") { + // Verifies DPP works when broadcast is disabled, triggering SubqueryExec. assume(icebergAvailable, "Iceberg not available") withTempIcebergDir { warehouseDir => val dimDir = new File(warehouseDir, "dim_parquet_subquery") @@ -2465,12 +2465,11 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { "spark.sql.catalog.subquery_cat" -> "org.apache.iceberg.spark.SparkCatalog", "spark.sql.catalog.subquery_cat.type" -> "hadoop", "spark.sql.catalog.subquery_cat.warehouse" -> warehouseDir.getAbsolutePath, - // Must disable AQE (like DynamicPartitionPruningV1SuiteAEOff) SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", - // Key configs to trigger SubqueryExec instead of SubqueryBroadcastExec SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { @@ -2508,18 +2507,31 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { spark.catalog.cacheTable("dim_subquery") sql("ANALYZE TABLE dim_subquery COMPUTE STATISTICS FOR COLUMNS id, date") - // NO broadcast hint - allows SubqueryExec to be used val query = """SELECT f.* FROM subquery_cat.db.fact_table f |JOIN dim_subquery d ON f.date = d.date AND d.id = 1 |ORDER BY f.id""".stripMargin val df = spark.sql(query) - val planStr = df.queryExecution.executedPlan.toString + val plan = df.queryExecution.executedPlan + val planStr = plan.toString assert( planStr.contains("dynamicpruning"), s"Expected dynamic pruning in plan but got:\n$planStr") + // Verify SubqueryExec is used in runtimeFilters (not SubqueryBroadcastExec) + val icebergScans = collect(plan) { case s: CometIcebergNativeScanExec => s } + assert(icebergScans.nonEmpty, "Expected CometIcebergNativeScanExec in plan") + val scan = icebergScans.head + val dppFilters = scan.originalPlan.runtimeFilters.collect { + case DynamicPruningExpression(e: InSubqueryExec) => e.plan + } + assert(dppFilters.nonEmpty, s"Expected DPP filters but found none") + val hasSubqueryExec = dppFilters.exists(_.isInstanceOf[SubqueryExec]) + assert( + hasSubqueryExec, + s"Expected SubqueryExec in DPP filters but got: ${dppFilters.map(_.getClass.getSimpleName)}") + checkSparkAnswer(df) spark.sql("DROP TABLE subquery_cat.db.fact_table") diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index a44ff37092..5c4ff8075f 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, He import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateMode, BloomFilterAggregate} import org.apache.spark.sql.comet._ import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} -import org.apache.spark.sql.execution.{CollectLimitExec, ProjectExec, SparkPlan, SQLExecution, UnionExec} +import org.apache.spark.sql.execution.{CollectLimitExec, ProjectExec, ReusedSubqueryExec, SparkPlan, SQLExecution, SubqueryExec, UnionExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} @@ -368,6 +368,69 @@ class CometExecSuite extends CometTestBase { planStr.contains("dynamicpruning"), s"Expected dynamic pruning in plan but got:\n$planStr") + // Verify SubqueryExec is in the plan + val plan = df.queryExecution.executedPlan + val subqueries = plan.collectWithSubqueries { case s: SubqueryExec => s } + assert(subqueries.nonEmpty, s"Expected SubqueryExec in plan but found none:\n$plan") + + checkSparkAnswer(df) + } + } + } + } + + test("DPP with native_datafusion scan - ReusedSubqueryExec (subquery reuse)") { + // Reproduces "Subquery reuse across the whole plan" from DynamicPartitionPruningSuite. + // When the same subquery is used multiple times, Spark wraps it in ReusedSubqueryExec. + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true") { + withTable("df1", "df2") { + spark + .range(100) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format("parquet") + .saveAsTable("df1") + + spark + .range(10) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format("parquet") + .saveAsTable("df2") + + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { + + // Query with scalar subquery + DPP join triggers ReusedSubqueryExec + val query = + """SELECT df1.id, df2.k + |FROM df1 JOIN df2 ON df1.k = df2.k + |WHERE df2.id < (SELECT max(id) FROM df2 WHERE id <= 2)""".stripMargin + + val df = sql(query) + val planStr = df.queryExecution.executedPlan.toString + assert( + planStr.contains("dynamicpruning"), + s"Expected dynamic pruning in plan but got:\n$planStr") + + // Verify ReusedSubqueryExec is in the plan + val plan = df.queryExecution.executedPlan + val reusedSubqueries = plan.collectWithSubqueries { case rs: ReusedSubqueryExec => + rs + } + assert( + reusedSubqueries.nonEmpty, + s"Expected ReusedSubqueryExec in plan but found none:\n$plan") + checkSparkAnswer(df) } } From b7eedd114c37e7814efc35d3e9272b50ed1d83ed Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 16:26:53 -0500 Subject: [PATCH 16/38] Fix format. --- .../test/scala/org/apache/comet/CometIcebergNativeSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index dc52a6df75..225bd2b50e 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -2526,7 +2526,7 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { val dppFilters = scan.originalPlan.runtimeFilters.collect { case DynamicPruningExpression(e: InSubqueryExec) => e.plan } - assert(dppFilters.nonEmpty, s"Expected DPP filters but found none") + assert(dppFilters.nonEmpty, "Expected DPP filters but found none") val hasSubqueryExec = dppFilters.exists(_.isInstanceOf[SubqueryExec]) assert( hasSubqueryExec, From e9dab0e7c6c28b45dc4ad373da5933dcfe91e8e1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 17:10:42 -0500 Subject: [PATCH 17/38] clean up tables in new tests --- .../test/scala/org/apache/comet/exec/CometExecSuite.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 5c4ff8075f..8b62b4bb64 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -207,6 +207,9 @@ class CometExecSuite extends CometTestBase { // multiple file partitions from one directory, so we check < 3 not == 1. val numPartitions = nativeScans.head.perPartitionData.length assert(numPartitions < 3, s"Expected DPP to prune partitions but got $numPartitions") + + spark.catalog.dropTempView("fact") + spark.catalog.dropTempView("dim") } } } @@ -277,6 +280,9 @@ class CometExecSuite extends CometTestBase { // We're filtering to region=US, category=A which is 1 partition. val numPartitions = nativeScans.head.perPartitionData.length assert(numPartitions < 6, s"Expected DPP to prune partitions but got $numPartitions") + + spark.catalog.dropTempView("fact") + spark.catalog.dropTempView("dim") } } } From b9b5bb88fc30e0313225647c2c808c32e811c692 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 19:09:43 -0500 Subject: [PATCH 18/38] update spark 3.5.8 diff --- dev/diffs/3.5.8.diff | 26 ++++++++++++++++---------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index beef445490..d6247f7bb2 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -410,28 +410,31 @@ index c4fb4fa943c..a04b23870a8 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..42eb9fd1cb7 100644 +index f33432ddb6f..665d414c61f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.plans.ExistenceJoin -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -262,6 +263,9 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -262,6 +263,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case s: CometScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child ++ } ++ case s: CometNativeScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child + } case _ => Nil } } -@@ -1027,7 +1031,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -441,7 +444,7 @@ index f33432ddb6f..42eb9fd1cb7 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1220,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1223,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -451,7 +454,7 @@ index f33432ddb6f..42eb9fd1cb7 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1429,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1432,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -461,7 +464,7 @@ index f33432ddb6f..42eb9fd1cb7 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1705,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1708,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -471,7 +474,7 @@ index f33432ddb6f..42eb9fd1cb7 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1737,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1740,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -3200,7 +3203,7 @@ index c63c748953f..7edca9c93a6 100644 implicit val formats = new DefaultFormats { override def dateFormatter = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala -index 52abd248f3a..7a199931a08 100644 +index 52abd248f3a..b4e096cae24 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive @@ -3211,12 +3214,15 @@ index 52abd248f3a..7a199931a08 100644 import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.hive.execution.HiveTableScanExec -@@ -35,6 +36,9 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase +@@ -35,6 +36,12 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase case s: FileSourceScanExec => s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: CometScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child ++ } ++ case s: CometNativeScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child + } case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child From 0e71307897a99e294a0fd31c0df7f20ae3b98f94 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 19:31:10 -0500 Subject: [PATCH 19/38] add bucketed DPP scan support --- .../spark/sql/comet/CometNativeScanExec.scala | 12 +++- .../sql/comet/shims/ShimCometScanExec.scala | 66 ++++++++++++++++++ .../sql/comet/shims/ShimCometScanExec.scala | 66 ++++++++++++++++++ .../sql/comet/shims/ShimCometScanExec.scala | 67 +++++++++++++++++++ 4 files changed, 210 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 4f7ceeacc2..4458466bc3 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -120,7 +120,17 @@ case class CometNativeScanExec( /** Get file partitions with DPP filtering applied. */ private def getFilePartitions(): Seq[FilePartition] = { - getDppFilteredFilePartitions(relation, partitionFilters, originalPlan.selectedPartitions) + if (bucketedScan) { + getDppFilteredBucketedFilePartitions( + relation, + partitionFilters, + originalPlan.selectedPartitions, + relation.bucketSpec.get, + optionalBucketSet, + optionalNumCoalescedBuckets) + } else { + getDppFilteredFilePartitions(relation, partitionFilters, originalPlan.selectedPartitions) + } } def commonData: Array[Byte] = serializedPartitionData._1 diff --git a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index c3e1a41261..4d77163b1f 100644 --- a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -22,12 +22,14 @@ package org.apache.spark.sql.comet.shims import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType +import org.apache.spark.util.collection.BitSet import org.apache.comet.shims.ShimFileFormat @@ -86,6 +88,70 @@ trait ShimCometScanExec { FilePartition.getFilePartitions(relation.sparkSession, splitFilesList, maxSplitBytes) } + /** + * Returns file partitions for bucketed tables after applying DPP filtering. Groups files by + * bucket ID to preserve bucket boundaries. + * + * Based on FileSourceScanExec.createBucketedReadRDD. + */ + protected def getDppFilteredBucketedFilePartitions( + relation: HadoopFsRelation, + partitionFilters: Seq[Expression], + selectedPartitions: Array[PartitionDirectory], + bucketSpec: BucketSpec, + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int]): Seq[FilePartition] = { + // First apply DPP filtering + val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) + val filteredPartitions = if (dynamicPartitionFilters.nonEmpty) { + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil) + selectedPartitions.filter(p => boundPredicate.eval(p.values)) + } else { + selectedPartitions + } + + // Group files by bucket ID + val filesGroupedToBuckets = filteredPartitions + .flatMap { p => + p.files.map(f => getPartitionedFile(f, p)) + } + .groupBy { f => + BucketingUtils + .getBucketId(f.toPath.getName) + .getOrElse(throw new IllegalStateException(s"Invalid bucket file: ${f.toPath}")) + } + + // Apply bucket pruning + val prunedFilesGroupedToBuckets = optionalBucketSet match { + case Some(bucketSet) => + filesGroupedToBuckets.filter { case (bucketId, _) => bucketSet.get(bucketId) } + case None => + filesGroupedToBuckets + } + + // Create file partitions - either coalesced or one per bucket + optionalNumCoalescedBuckets match { + case Some(numCoalescedBuckets) => + val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) + Seq.tabulate(numCoalescedBuckets) { bucketId => + val files = + coalescedBuckets.get(bucketId).map(_.values.flatten.toArray).getOrElse(Array.empty) + FilePartition(bucketId, files) + } + case None => + Seq.tabulate(bucketSpec.numBuckets) { bucketId => + FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + } + } + } + private def isDynamicPruningFilter(e: Expression): Boolean = e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) diff --git a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 33b1b7b18d..3c3d7397f3 100644 --- a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SPARK_VERSION_SHORT import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources._ @@ -32,6 +33,7 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.VersionUtils +import org.apache.spark.util.collection.BitSet trait ShimCometScanExec { def wrapped: FileSourceScanExec @@ -88,6 +90,70 @@ trait ShimCometScanExec { FilePartition.getFilePartitions(relation.sparkSession, splitFilesList, maxSplitBytes) } + /** + * Returns file partitions for bucketed tables after applying DPP filtering. Groups files by + * bucket ID to preserve bucket boundaries. + * + * Based on FileSourceScanExec.createBucketedReadRDD. + */ + protected def getDppFilteredBucketedFilePartitions( + relation: HadoopFsRelation, + partitionFilters: Seq[Expression], + selectedPartitions: Array[PartitionDirectory], + bucketSpec: BucketSpec, + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int]): Seq[FilePartition] = { + // First apply DPP filtering + val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) + val filteredPartitions = if (dynamicPartitionFilters.nonEmpty) { + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil) + selectedPartitions.filter(p => boundPredicate.eval(p.values)) + } else { + selectedPartitions + } + + // Group files by bucket ID + val filesGroupedToBuckets = filteredPartitions + .flatMap { p => + p.files.map(f => getPartitionedFile(f, p)) + } + .groupBy { f => + BucketingUtils + .getBucketId(f.toPath.getName) + .getOrElse(throw new IllegalStateException(s"Invalid bucket file: ${f.toPath}")) + } + + // Apply bucket pruning + val prunedFilesGroupedToBuckets = optionalBucketSet match { + case Some(bucketSet) => + filesGroupedToBuckets.filter { case (bucketId, _) => bucketSet.get(bucketId) } + case None => + filesGroupedToBuckets + } + + // Create file partitions - either coalesced or one per bucket + optionalNumCoalescedBuckets match { + case Some(numCoalescedBuckets) => + val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) + Seq.tabulate(numCoalescedBuckets) { bucketId => + val files = + coalescedBuckets.get(bucketId).map(_.values.flatten.toArray).getOrElse(Array.empty) + FilePartition(bucketId, files) + } + case None => + Seq.tabulate(bucketSpec.numBuckets) { bucketId => + FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + } + } + } + private def isDynamicPruningFilter(e: Expression): Boolean = e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 1c9f240242..e330ab658f 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -22,12 +22,14 @@ package org.apache.spark.sql.comet.shims import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, FileSourceConstantMetadataAttribute, Literal, Predicate} import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil, ScalarSubquery, ScanFileListing} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType +import org.apache.spark.util.collection.BitSet trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { def wrapped: FileSourceScanExec @@ -85,6 +87,71 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { FilePartition.getFilePartitions(relation.sparkSession, splitFilesList, maxSplitBytes) } + /** + * Returns file partitions for bucketed tables after applying DPP filtering. Groups files by + * bucket ID to preserve bucket boundaries. + * + * Based on FileSourceScanExec.createBucketedReadRDD. + */ + protected def getDppFilteredBucketedFilePartitions( + relation: HadoopFsRelation, + partitionFilters: Seq[Expression], + selectedPartitions: ScanFileListing, + bucketSpec: BucketSpec, + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int]): Seq[FilePartition] = { + // First apply DPP filtering + val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) + val filteredListing = if (dynamicPartitionFilters.nonEmpty) { + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil) + selectedPartitions.filterAndPruneFiles(boundPredicate, Seq.empty) + } else { + selectedPartitions + } + + // Group files by bucket ID + val filesGroupedToBuckets = filteredListing.filePartitionIterator + .flatMap { p => + p.files.map(f => getPartitionedFile(f, p)) + } + .toSeq + .groupBy { f => + BucketingUtils + .getBucketId(f.toPath.getName) + .getOrElse(throw new IllegalStateException(s"Invalid bucket file: ${f.toPath}")) + } + + // Apply bucket pruning + val prunedFilesGroupedToBuckets = optionalBucketSet match { + case Some(bucketSet) => + filesGroupedToBuckets.filter { case (bucketId, _) => bucketSet.get(bucketId) } + case None => + filesGroupedToBuckets + } + + // Create file partitions - either coalesced or one per bucket + optionalNumCoalescedBuckets match { + case Some(numCoalescedBuckets) => + val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) + Seq.tabulate(numCoalescedBuckets) { bucketId => + val files = + coalescedBuckets.get(bucketId).map(_.values.flatten.toArray).getOrElse(Array.empty) + FilePartition(bucketId, files) + } + case None => + Seq.tabulate(bucketSpec.numBuckets) { bucketId => + FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + } + } + } + private def isDynamicPruningFilter(e: Expression): Boolean = e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) From fa6fd1a575cd24cb43a191a43ec943910397ddb8 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 19:40:29 -0500 Subject: [PATCH 20/38] update spark 4.0 shim --- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index e330ab658f..afad089b33 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -117,9 +117,11 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { } // Group files by bucket ID + // Note: filePartitionIterator returns ListingPartition in Spark 4.0, so we inline + // the PartitionedFile creation instead of using getPartitionedFile(f, PartitionDirectory) val filesGroupedToBuckets = filteredListing.filePartitionIterator .flatMap { p => - p.files.map(f => getPartitionedFile(f, p)) + p.files.map(f => PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values, 0, f.getLen)) } .toSeq .groupBy { f => From ed50be222e05b776a37372ba409c8e13f41bff6e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 19:52:26 -0500 Subject: [PATCH 21/38] make format --- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index afad089b33..cfd1b33610 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -121,7 +121,8 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { // the PartitionedFile creation instead of using getPartitionedFile(f, PartitionDirectory) val filesGroupedToBuckets = filteredListing.filePartitionIterator .flatMap { p => - p.files.map(f => PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values, 0, f.getLen)) + p.files.map(f => + PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values, 0, f.getLen)) } .toSeq .groupBy { f => From 48ba80a4c3b6ab2676ee223d773d7984b86cc200 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 20:20:53 -0500 Subject: [PATCH 22/38] fix shims --- .../apache/spark/sql/comet/shims/ShimCometScanExec.scala | 9 +++++++-- .../apache/spark/sql/comet/shims/ShimCometScanExec.scala | 9 +++++++-- .../apache/spark/sql/comet/shims/ShimCometScanExec.scala | 9 +++++++-- 3 files changed, 21 insertions(+), 6 deletions(-) diff --git a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 4d77163b1f..285190be46 100644 --- a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -142,12 +142,17 @@ trait ShimCometScanExec { val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) Seq.tabulate(numCoalescedBuckets) { bucketId => val files = - coalescedBuckets.get(bucketId).map(_.values.flatten.toArray).getOrElse(Array.empty) + coalescedBuckets + .get(bucketId) + .map(_.values.flatten.toArray) + .getOrElse(Array.empty[PartitionedFile]) FilePartition(bucketId, files) } case None => Seq.tabulate(bucketSpec.numBuckets) { bucketId => - FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + FilePartition( + bucketId, + prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty).toArray) } } } diff --git a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 3c3d7397f3..e412ee6464 100644 --- a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -144,12 +144,17 @@ trait ShimCometScanExec { val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) Seq.tabulate(numCoalescedBuckets) { bucketId => val files = - coalescedBuckets.get(bucketId).map(_.values.flatten.toArray).getOrElse(Array.empty) + coalescedBuckets + .get(bucketId) + .map(_.values.flatten.toArray) + .getOrElse(Array.empty[PartitionedFile]) FilePartition(bucketId, files) } case None => Seq.tabulate(bucketSpec.numBuckets) { bucketId => - FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + FilePartition( + bucketId, + prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty).toArray) } } } diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index cfd1b33610..1c443e5d13 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -145,12 +145,17 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) Seq.tabulate(numCoalescedBuckets) { bucketId => val files = - coalescedBuckets.get(bucketId).map(_.values.flatten.toArray).getOrElse(Array.empty) + coalescedBuckets + .get(bucketId) + .map(_.values.flatten.toArray) + .getOrElse(Array.empty[PartitionedFile]) FilePartition(bucketId, files) } case None => Seq.tabulate(bucketSpec.numBuckets) { bucketId => - FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + FilePartition( + bucketId, + prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty).toArray) } } } From cd6539c2831dd2b2bd5630cca8312834d551ab69 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 20:29:55 -0500 Subject: [PATCH 23/38] fix shims --- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 2 +- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 285190be46..710f0ebfc5 100644 --- a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -152,7 +152,7 @@ trait ShimCometScanExec { Seq.tabulate(bucketSpec.numBuckets) { bucketId => FilePartition( bucketId, - prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty).toArray) + prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty[PartitionedFile])) } } } diff --git a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index e412ee6464..695d73707e 100644 --- a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -154,7 +154,7 @@ trait ShimCometScanExec { Seq.tabulate(bucketSpec.numBuckets) { bucketId => FilePartition( bucketId, - prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty).toArray) + prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty[PartitionedFile])) } } } From 9b00cc274fd0c8d979b7535689a40b0ecb792dcc Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Feb 2026 21:37:59 -0500 Subject: [PATCH 24/38] fix canonicalization? --- .../scala/org/apache/spark/sql/comet/CometNativeScanExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 4458466bc3..a4d93cc114 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -191,7 +191,7 @@ case class CometNativeScanExec( QueryPlan.normalizePredicates(dataFilters, output), None, disableBucketedScan, - null, // Don't need originalPlan for canonicalization + originalPlan.doCanonicalize(), SerializedPlan(None)) } From 704bd2de3010dead8067c704256a4ecd2e090007 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Feb 2026 06:56:15 -0500 Subject: [PATCH 25/38] Update diffs --- .../q10.native_datafusion/extended.txt | 2 +- .../q11.native_datafusion/extended.txt | 4 +- .../q14a.native_datafusion/extended.txt | 16 ++-- .../q14b.native_datafusion/extended.txt | 8 +- .../q19.native_datafusion/extended.txt | 2 +- .../q25.native_datafusion/extended.txt | 2 +- .../q29.native_datafusion/extended.txt | 2 +- .../q31.native_datafusion/extended.txt | 6 +- .../q33.native_datafusion/extended.txt | 2 +- .../q35.native_datafusion/extended.txt | 2 +- .../q38.native_datafusion/extended.txt | 4 +- .../q4.native_datafusion/extended.txt | 8 +- .../q49.native_datafusion/extended.txt | 8 +- .../q5.native_datafusion/extended.txt | 8 +- .../q51.native_datafusion/extended.txt | 2 +- .../q56.native_datafusion/extended.txt | 2 +- .../q58.native_datafusion/extended.txt | 4 +- .../q60.native_datafusion/extended.txt | 2 +- .../q61.native_datafusion/extended.txt | 4 +- .../q62.native_datafusion/extended.txt | 2 +- .../q64.native_datafusion/extended.txt | 8 +- .../q66.native_datafusion/extended.txt | 2 +- .../q69.native_datafusion/extended.txt | 2 +- .../q72.native_datafusion/extended.txt | 2 +- .../q74.native_datafusion/extended.txt | 4 +- .../q75.native_datafusion/extended.txt | 16 ++-- .../q77.native_datafusion/extended.txt | 10 +-- .../q78.native_datafusion/extended.txt | 6 +- .../q80.native_datafusion/extended.txt | 10 +-- .../q83.native_datafusion/extended.txt | 4 +- .../q87.native_datafusion/extended.txt | 4 +- .../q99.native_datafusion/extended.txt | 2 +- .../q10.native_datafusion/extended.txt | 2 +- .../q11.native_datafusion/extended.txt | 4 +- .../q14a.native_datafusion/extended.txt | 16 ++-- .../q14b.native_datafusion/extended.txt | 8 +- .../q19.native_datafusion/extended.txt | 2 +- .../q25.native_datafusion/extended.txt | 2 +- .../q29.native_datafusion/extended.txt | 2 +- .../q31.native_datafusion/extended.txt | 6 +- .../q33.native_datafusion/extended.txt | 2 +- .../q35.native_datafusion/extended.txt | 2 +- .../q38.native_datafusion/extended.txt | 4 +- .../q4.native_datafusion/extended.txt | 8 +- .../q49.native_datafusion/extended.txt | 8 +- .../q5.native_datafusion/extended.txt | 8 +- .../q51.native_datafusion/extended.txt | 2 +- .../q56.native_datafusion/extended.txt | 2 +- .../q58.native_datafusion/extended.txt | 4 +- .../q60.native_datafusion/extended.txt | 2 +- .../q61.native_datafusion/extended.txt | 4 +- .../q62.native_datafusion/extended.txt | 2 +- .../q64.native_datafusion/extended.txt | 8 +- .../q66.native_datafusion/extended.txt | 2 +- .../q69.native_datafusion/extended.txt | 2 +- .../q72.native_datafusion/extended.txt | 2 +- .../q74.native_datafusion/extended.txt | 4 +- .../q75.native_datafusion/extended.txt | 16 ++-- .../q77.native_datafusion/extended.txt | 10 +-- .../q78.native_datafusion/extended.txt | 6 +- .../q80.native_datafusion/extended.txt | 10 +-- .../q83.ansi.native_datafusion/extended.txt | 4 +- .../q87.native_datafusion/extended.txt | 4 +- .../q99.native_datafusion/extended.txt | 2 +- .../q10.native_datafusion/extended.txt | 2 +- .../q11.native_datafusion/extended.txt | 4 +- .../q14a.native_datafusion/extended.txt | 16 ++-- .../q14b.native_datafusion/extended.txt | 8 +- .../q19.native_datafusion/extended.txt | 2 +- .../q25.native_datafusion/extended.txt | 2 +- .../q29.native_datafusion/extended.txt | 2 +- .../q31.native_datafusion/extended.txt | 6 +- .../q35.native_datafusion/extended.txt | 2 +- .../q38.native_datafusion/extended.txt | 4 +- .../q4.native_datafusion/extended.txt | 8 +- .../q49.native_datafusion/extended.txt | 8 +- .../q5.native_datafusion/extended.txt | 8 +- .../q51.native_datafusion/extended.txt | 2 +- .../q56.native_datafusion/extended.txt | 2 +- .../q58.native_datafusion/extended.txt | 2 +- .../q60.native_datafusion/extended.txt | 2 +- .../q61.native_datafusion/extended.txt | 4 +- .../q62.native_datafusion/extended.txt | 2 +- .../q64.native_datafusion/extended.txt | 8 +- .../q66.native_datafusion/extended.txt | 2 +- .../q69.native_datafusion/extended.txt | 2 +- .../q72.native_datafusion/extended.txt | 2 +- .../q74.native_datafusion/extended.txt | 4 +- .../q75.native_datafusion/extended.txt | 16 ++-- .../q77.native_datafusion/extended.txt | 10 +-- .../q78.native_datafusion/extended.txt | 6 +- .../q80.native_datafusion/extended.txt | 10 +-- .../q83.native_datafusion/extended.txt | 2 +- .../q87.native_datafusion/extended.txt | 4 +- .../q99.native_datafusion/extended.txt | 2 +- .../q11.native_datafusion/extended.txt | 4 +- .../q14.native_datafusion/extended.txt | 8 +- .../q14a.native_datafusion/extended.txt | 80 +++++++++---------- .../q18a.native_datafusion/extended.txt | 2 +- .../q35.native_datafusion/extended.txt | 2 +- .../q49.native_datafusion/extended.txt | 8 +- .../q51a.native_datafusion/extended.txt | 8 +- .../q5a.native_datafusion/extended.txt | 24 +++--- .../q64.native_datafusion/extended.txt | 8 +- .../q72.native_datafusion/extended.txt | 2 +- .../q74.native_datafusion/extended.txt | 4 +- .../q75.native_datafusion/extended.txt | 16 ++-- .../q77a.native_datafusion/extended.txt | 30 +++---- .../q78.native_datafusion/extended.txt | 6 +- .../q80a.native_datafusion/extended.txt | 30 +++---- .../q11.native_datafusion/extended.txt | 4 +- .../q14.native_datafusion/extended.txt | 8 +- .../q14a.native_datafusion/extended.txt | 80 +++++++++---------- .../q18a.native_datafusion/extended.txt | 2 +- .../q35.native_datafusion/extended.txt | 2 +- .../q49.native_datafusion/extended.txt | 8 +- .../q51a.native_datafusion/extended.txt | 8 +- .../q5a.native_datafusion/extended.txt | 24 +++--- .../q64.native_datafusion/extended.txt | 8 +- .../q72.native_datafusion/extended.txt | 2 +- .../q74.native_datafusion/extended.txt | 4 +- .../q75.native_datafusion/extended.txt | 16 ++-- .../q77a.native_datafusion/extended.txt | 30 +++---- .../q78.native_datafusion/extended.txt | 6 +- .../q80a.native_datafusion/extended.txt | 30 +++---- .../q11.native_datafusion/extended.txt | 4 +- .../q14.native_datafusion/extended.txt | 8 +- .../q14a.native_datafusion/extended.txt | 80 +++++++++---------- .../q18a.native_datafusion/extended.txt | 2 +- .../q35.native_datafusion/extended.txt | 2 +- .../q49.native_datafusion/extended.txt | 8 +- .../q51a.native_datafusion/extended.txt | 8 +- .../q5a.native_datafusion/extended.txt | 24 +++--- .../q64.native_datafusion/extended.txt | 8 +- .../q72.native_datafusion/extended.txt | 2 +- .../q74.native_datafusion/extended.txt | 4 +- .../q75.native_datafusion/extended.txt | 16 ++-- .../q77a.native_datafusion/extended.txt | 30 +++---- .../q78.native_datafusion/extended.txt | 6 +- .../q80a.native_datafusion/extended.txt | 30 +++---- 140 files changed, 585 insertions(+), 585 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt index 55ce155c89..6e6f311b7d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt @@ -64,7 +64,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -86,7 +86,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt index e16c7013b4..ab185e742b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt @@ -124,7 +124,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -198,7 +198,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -228,7 +228,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -291,7 +291,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -365,7 +365,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -458,7 +458,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -532,7 +532,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt index 4b85046c30..bf97aa989e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt @@ -124,7 +124,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -198,7 +198,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -297,7 +297,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -371,7 +371,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt index d76549df2e..fbe6798e3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/extended.txt @@ -33,6 +33,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometNativeScan parquet spark_catalog.default.store Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt index 6b601f21a0..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt @@ -62,6 +62,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt index 6b601f21a0..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt @@ -62,6 +62,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt index 3fc7ae5a54..680121f013 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt @@ -80,7 +80,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -101,7 +101,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -122,7 +122,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt index 57033e3622..89aa813e43 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt @@ -80,7 +80,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt index d976fd7f57..1a323c6c34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt @@ -37,7 +37,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -61,7 +61,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt index 6acf46921e..2a5b362a3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt @@ -67,7 +67,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- CometBroadcastExchange : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -112,7 +112,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -134,7 +134,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt index ff7ad3b219..084971ac28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt @@ -54,7 +54,7 @@ CometColumnarToRow : : :- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -86,7 +86,7 @@ CometColumnarToRow : :- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -95,7 +95,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt index 45e5c0f0e9..1472b14332 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : :- CometUnion : : : :- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -72,7 +72,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.catalog_page +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -110,6 +110,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_site Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt index 40b425a184..d60ef8792f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometNativeScan parquet spark_catalog.default.store_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt index 59e112c499..499942b26c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt @@ -82,7 +82,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt index 72bd7e9cba..f86ed28206 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt @@ -58,7 +58,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -103,7 +103,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt index 59e112c499..499942b26c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt @@ -82,7 +82,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt index 2d4c85b5a5..a8cafce32e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt @@ -42,7 +42,7 @@ Project : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -84,7 +84,7 @@ Project : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.customer_address +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt index 0bfc07426a..52d7828b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometNativeScan parquet spark_catalog.default.web_site +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt index 8aa248f6ca..f1364542ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt @@ -115,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : +- CometNativeScan parquet spark_catalog.default.income_band : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -235,10 +235,10 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometNativeScan parquet spark_catalog.default.income_band +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt index 3d8af2a097..209f7a9f0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt index 67d364800e..8aa58425a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt @@ -47,7 +47,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt index 7a49cad749..5e78f07bd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt index 33a79ee1f7..5d93f1897b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ CometColumnarToRow : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort @@ -75,7 +75,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -92,7 +92,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.web_returns +- CometSort +- CometExchange +- CometFilter @@ -139,7 +139,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometProject +- CometSortMergeJoin :- CometSort @@ -166,7 +166,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -183,6 +183,6 @@ CometColumnarToRow +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_returns + +- CometNativeScan parquet spark_catalog.default.web_returns Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt index 4b08389eae..e4912d3273 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -101,7 +101,7 @@ TakeOrderedAndProject : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- CometBroadcastExchange +- CometHashAggregate +- CometExchange @@ -124,7 +124,7 @@ TakeOrderedAndProject :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -137,6 +137,6 @@ TakeOrderedAndProject : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_page Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt index c253b8aff8..fa87e0b01d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject : : :- CometSort : : : +- CometExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -53,7 +53,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt index 5ffd41f4ba..bb1998687e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt @@ -76,7 +76,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,7 +84,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -109,7 +109,7 @@ CometColumnarToRow : : : : :- CometSort : : : : : +- CometExchange : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -120,7 +120,7 @@ CometColumnarToRow : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -128,7 +128,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_site : +- CometBroadcastExchange : +- CometProject : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt index dc88a9234f..d10feb6fb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt @@ -54,7 +54,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -96,7 +96,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt index 05d07c8d36..5b119a3926 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt @@ -39,7 +39,7 @@ HashAggregate : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -64,7 +64,7 @@ HashAggregate :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt index 6059603410..c075163849 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometNativeScan parquet spark_catalog.default.call_center +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt index 55ce155c89..6e6f311b7d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt @@ -64,7 +64,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -86,7 +86,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt index e16c7013b4..ab185e742b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt @@ -124,7 +124,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -198,7 +198,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -228,7 +228,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -291,7 +291,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -365,7 +365,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -458,7 +458,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -532,7 +532,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt index 6d6eda80ae..2b89b148f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt @@ -129,7 +129,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -203,7 +203,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -312,7 +312,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -386,7 +386,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt index d76549df2e..fbe6798e3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/extended.txt @@ -33,6 +33,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometNativeScan parquet spark_catalog.default.store Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt index 6b601f21a0..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt @@ -62,6 +62,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt index 6b601f21a0..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt @@ -62,6 +62,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt index 3fc7ae5a54..680121f013 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt @@ -80,7 +80,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -101,7 +101,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -122,7 +122,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt index 57033e3622..89aa813e43 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt @@ -80,7 +80,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt index d976fd7f57..1a323c6c34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt @@ -37,7 +37,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -61,7 +61,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt index 6acf46921e..2a5b362a3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt @@ -67,7 +67,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- CometBroadcastExchange : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -112,7 +112,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -134,7 +134,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt index ff7ad3b219..084971ac28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt @@ -54,7 +54,7 @@ CometColumnarToRow : : :- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -86,7 +86,7 @@ CometColumnarToRow : :- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -95,7 +95,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt index 45e5c0f0e9..1472b14332 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : :- CometUnion : : : :- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -72,7 +72,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.catalog_page +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -110,6 +110,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_site Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt index 40b425a184..d60ef8792f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometNativeScan parquet spark_catalog.default.store_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt index 59e112c499..499942b26c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt @@ -82,7 +82,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt index 7f74b937ac..ceb98a1e98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt @@ -64,7 +64,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -115,7 +115,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt index 59e112c499..499942b26c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt @@ -82,7 +82,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt index 2d4c85b5a5..a8cafce32e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt @@ -42,7 +42,7 @@ Project : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -84,7 +84,7 @@ Project : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.customer_address +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt index 0bfc07426a..52d7828b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometNativeScan parquet spark_catalog.default.web_site +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt index 8aa248f6ca..f1364542ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt @@ -115,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : +- CometNativeScan parquet spark_catalog.default.income_band : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -235,10 +235,10 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometNativeScan parquet spark_catalog.default.income_band +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt index 3d8af2a097..209f7a9f0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt index 67d364800e..8aa58425a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt @@ -47,7 +47,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt index 7a49cad749..5e78f07bd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt index 33a79ee1f7..5d93f1897b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ CometColumnarToRow : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort @@ -75,7 +75,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -92,7 +92,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.web_returns +- CometSort +- CometExchange +- CometFilter @@ -139,7 +139,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometProject +- CometSortMergeJoin :- CometSort @@ -166,7 +166,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -183,6 +183,6 @@ CometColumnarToRow +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_returns + +- CometNativeScan parquet spark_catalog.default.web_returns Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt index 4b08389eae..e4912d3273 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -101,7 +101,7 @@ TakeOrderedAndProject : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- CometBroadcastExchange +- CometHashAggregate +- CometExchange @@ -124,7 +124,7 @@ TakeOrderedAndProject :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -137,6 +137,6 @@ TakeOrderedAndProject : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_page Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt index c253b8aff8..fa87e0b01d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject : : :- CometSort : : : +- CometExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -53,7 +53,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt index 5ffd41f4ba..bb1998687e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt @@ -76,7 +76,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,7 +84,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -109,7 +109,7 @@ CometColumnarToRow : : : : :- CometSort : : : : : +- CometExchange : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -120,7 +120,7 @@ CometColumnarToRow : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -128,7 +128,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_site : +- CometBroadcastExchange : +- CometProject : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt index dc88a9234f..d10feb6fb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt @@ -54,7 +54,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -96,7 +96,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt index 05d07c8d36..5b119a3926 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt @@ -39,7 +39,7 @@ HashAggregate : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -64,7 +64,7 @@ HashAggregate :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt index 6059603410..c075163849 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometNativeScan parquet spark_catalog.default.call_center +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt index 55ce155c89..6e6f311b7d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt @@ -64,7 +64,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -86,7 +86,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index e16c7013b4..ab185e742b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt @@ -124,7 +124,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -198,7 +198,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -228,7 +228,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -291,7 +291,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -365,7 +365,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -458,7 +458,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -532,7 +532,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt index 4b85046c30..bf97aa989e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt @@ -124,7 +124,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -198,7 +198,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -297,7 +297,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -371,7 +371,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt index d76549df2e..fbe6798e3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt @@ -33,6 +33,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometNativeScan parquet spark_catalog.default.store Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index 6b601f21a0..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt @@ -62,6 +62,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt index 6b601f21a0..c2f112b9bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt @@ -62,6 +62,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index 3fc7ae5a54..680121f013 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt @@ -80,7 +80,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -101,7 +101,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -122,7 +122,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt index d976fd7f57..1a323c6c34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt @@ -37,7 +37,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -61,7 +61,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index 6acf46921e..2a5b362a3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt @@ -67,7 +67,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : : +- CometBroadcastExchange : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -89,7 +89,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -112,7 +112,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -134,7 +134,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index ff7ad3b219..084971ac28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -54,7 +54,7 @@ CometColumnarToRow : : :- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -86,7 +86,7 @@ CometColumnarToRow : :- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -95,7 +95,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt index 45e5c0f0e9..1472b14332 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : :- CometUnion : : : :- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -58,7 +58,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -72,7 +72,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.catalog_page +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -110,6 +110,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_site Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index 40b425a184..d60ef8792f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometNativeScan parquet spark_catalog.default.store_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt index 59e112c499..499942b26c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt @@ -82,7 +82,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt index bfca848535..6a89786dc2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt @@ -99,7 +99,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index 59e112c499..499942b26c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt @@ -82,7 +82,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index 2d4c85b5a5..a8cafce32e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt @@ -42,7 +42,7 @@ Project : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.customer_address : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -84,7 +84,7 @@ Project : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.customer_address +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt index 0bfc07426a..52d7828b06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometNativeScan parquet spark_catalog.default.web_site +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt index 8aa248f6ca..f1364542ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt @@ -115,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : +- CometNativeScan parquet spark_catalog.default.income_band : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -235,10 +235,10 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometNativeScan parquet spark_catalog.default.income_band +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index 3d8af2a097..209f7a9f0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt @@ -49,7 +49,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index 67d364800e..8aa58425a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt @@ -47,7 +47,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt index 7a49cad749..5e78f07bd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt index 33a79ee1f7..5d93f1897b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ CometColumnarToRow : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort @@ -75,7 +75,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -92,7 +92,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.web_returns +- CometSort +- CometExchange +- CometFilter @@ -139,7 +139,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometProject +- CometSortMergeJoin :- CometSort @@ -166,7 +166,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -183,6 +183,6 @@ CometColumnarToRow +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_returns + +- CometNativeScan parquet spark_catalog.default.web_returns Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt index 4b08389eae..e4912d3273 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt @@ -39,7 +39,7 @@ TakeOrderedAndProject : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -101,7 +101,7 @@ TakeOrderedAndProject : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -114,7 +114,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- CometBroadcastExchange +- CometHashAggregate +- CometExchange @@ -124,7 +124,7 @@ TakeOrderedAndProject :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -137,6 +137,6 @@ TakeOrderedAndProject : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_page Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt index c253b8aff8..fa87e0b01d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject : : :- CometSort : : : +- CometExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -53,7 +53,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt index 5ffd41f4ba..bb1998687e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt @@ -76,7 +76,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,7 +84,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -109,7 +109,7 @@ CometColumnarToRow : : : : :- CometSort : : : : : +- CometExchange : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -120,7 +120,7 @@ CometColumnarToRow : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -128,7 +128,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_site : +- CometBroadcastExchange : +- CometProject : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index 23598e5776..d10feb6fb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt @@ -96,7 +96,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index 05d07c8d36..5b119a3926 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt @@ -39,7 +39,7 @@ HashAggregate : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -64,7 +64,7 @@ HashAggregate :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt index 6059603410..c075163849 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt @@ -22,7 +22,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.ship_mode : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometNativeScan parquet spark_catalog.default.call_center +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt index 4b85046c30..bf97aa989e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt @@ -124,7 +124,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -198,7 +198,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -297,7 +297,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -371,7 +371,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt index 367fe17ed9..56a44482b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt @@ -126,7 +126,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -200,7 +200,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -229,7 +229,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -292,7 +292,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -366,7 +366,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -458,7 +458,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -532,7 +532,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -675,7 +675,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -749,7 +749,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -778,7 +778,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -841,7 +841,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -915,7 +915,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -944,7 +944,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1007,7 +1007,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1081,7 +1081,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1224,7 +1224,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1327,7 +1327,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1390,7 +1390,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1464,7 +1464,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1493,7 +1493,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1556,7 +1556,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1630,7 +1630,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1773,7 +1773,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1847,7 +1847,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1876,7 +1876,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1939,7 +1939,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2013,7 +2013,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2042,7 +2042,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2105,7 +2105,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2179,7 +2179,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2322,7 +2322,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2396,7 +2396,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2425,7 +2425,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2488,7 +2488,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2562,7 +2562,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2591,7 +2591,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2654,7 +2654,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2728,7 +2728,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt index f74b47bf8a..7b433fec06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt @@ -229,6 +229,6 @@ CometColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt index ff7ad3b219..084971ac28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt @@ -54,7 +54,7 @@ CometColumnarToRow : : :- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -86,7 +86,7 @@ CometColumnarToRow : :- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -95,7 +95,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt index 6714125d8e..3949c10caa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt @@ -83,7 +83,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -198,7 +198,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -221,7 +221,7 @@ TakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometNativeScan parquet spark_catalog.default.store_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt index 74efb6d851..88239ea6fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt @@ -52,7 +52,7 @@ CometColumnarToRow : : : :- CometUnion : : : : :- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -61,7 +61,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -75,7 +75,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -113,7 +113,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_site :- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -165,7 +165,7 @@ CometColumnarToRow : : : :- CometUnion : : : : :- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -174,7 +174,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -188,7 +188,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -226,7 +226,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_site +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -278,7 +278,7 @@ CometColumnarToRow : : :- CometUnion : : : :- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -287,7 +287,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -301,7 +301,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.catalog_page +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -339,6 +339,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_site Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt index 8aa248f6ca..f1364542ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt @@ -115,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : +- CometNativeScan parquet spark_catalog.default.income_band : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -235,10 +235,10 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometNativeScan parquet spark_catalog.default.income_band +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt index 7a49cad749..5e78f07bd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt index 33a79ee1f7..5d93f1897b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ CometColumnarToRow : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort @@ -75,7 +75,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -92,7 +92,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.web_returns +- CometSort +- CometExchange +- CometFilter @@ -139,7 +139,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometProject +- CometSortMergeJoin :- CometSort @@ -166,7 +166,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -183,6 +183,6 @@ CometColumnarToRow +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_returns + +- CometNativeScan parquet spark_catalog.default.web_returns Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt index e18882dfff..03ba668f90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt @@ -43,7 +43,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -105,7 +105,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -118,7 +118,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_page : +- CometBroadcastExchange : +- CometHashAggregate : +- CometExchange @@ -128,7 +128,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -141,7 +141,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -185,7 +185,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -247,7 +247,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -260,7 +260,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_page : +- CometBroadcastExchange : +- CometHashAggregate : +- CometExchange @@ -270,7 +270,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -283,7 +283,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -327,7 +327,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -389,7 +389,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -402,7 +402,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- CometBroadcastExchange +- CometHashAggregate +- CometExchange @@ -412,7 +412,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -425,6 +425,6 @@ CometColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_page Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt index c253b8aff8..fa87e0b01d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject : : :- CometSort : : : +- CometExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -53,7 +53,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt index b66eb1ce18..383373ab1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt @@ -79,7 +79,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -87,7 +87,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -112,7 +112,7 @@ CometColumnarToRow : : : : : :- CometSort : : : : : : +- CometExchange : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -123,7 +123,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -131,7 +131,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.web_site : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -218,7 +218,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -226,7 +226,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -251,7 +251,7 @@ CometColumnarToRow : : : : : :- CometSort : : : : : : +- CometExchange : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -262,7 +262,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -270,7 +270,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.web_site : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -357,7 +357,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -365,7 +365,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -390,7 +390,7 @@ CometColumnarToRow : : : : :- CometSort : : : : : +- CometExchange : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -401,7 +401,7 @@ CometColumnarToRow : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -409,7 +409,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_site : +- CometBroadcastExchange : +- CometProject : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt index 6d6eda80ae..2b89b148f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt @@ -129,7 +129,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -203,7 +203,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -312,7 +312,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -386,7 +386,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt index 367fe17ed9..56a44482b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt @@ -126,7 +126,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -200,7 +200,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -229,7 +229,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -292,7 +292,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -366,7 +366,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -458,7 +458,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -532,7 +532,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -675,7 +675,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -749,7 +749,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -778,7 +778,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -841,7 +841,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -915,7 +915,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -944,7 +944,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1007,7 +1007,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1081,7 +1081,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1224,7 +1224,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1327,7 +1327,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1390,7 +1390,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1464,7 +1464,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1493,7 +1493,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1556,7 +1556,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1630,7 +1630,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1773,7 +1773,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1847,7 +1847,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1876,7 +1876,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1939,7 +1939,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2013,7 +2013,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2042,7 +2042,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2105,7 +2105,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2179,7 +2179,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2322,7 +2322,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2396,7 +2396,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2425,7 +2425,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2488,7 +2488,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2562,7 +2562,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2591,7 +2591,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2654,7 +2654,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2728,7 +2728,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt index f74b47bf8a..7b433fec06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt @@ -229,6 +229,6 @@ CometColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt index ff7ad3b219..084971ac28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt @@ -54,7 +54,7 @@ CometColumnarToRow : : :- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -86,7 +86,7 @@ CometColumnarToRow : :- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -95,7 +95,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt index 6714125d8e..3949c10caa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt @@ -83,7 +83,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -198,7 +198,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -221,7 +221,7 @@ TakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometNativeScan parquet spark_catalog.default.store_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt index 74efb6d851..88239ea6fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt @@ -52,7 +52,7 @@ CometColumnarToRow : : : :- CometUnion : : : : :- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -61,7 +61,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -75,7 +75,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -113,7 +113,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_site :- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -165,7 +165,7 @@ CometColumnarToRow : : : :- CometUnion : : : : :- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -174,7 +174,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -188,7 +188,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -226,7 +226,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_site +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -278,7 +278,7 @@ CometColumnarToRow : : :- CometUnion : : : :- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -287,7 +287,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -301,7 +301,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.catalog_page +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -339,6 +339,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_site Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt index 8aa248f6ca..f1364542ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt @@ -115,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : +- CometNativeScan parquet spark_catalog.default.income_band : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -235,10 +235,10 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometNativeScan parquet spark_catalog.default.income_band +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt index 7a49cad749..5e78f07bd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt index 33a79ee1f7..5d93f1897b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ CometColumnarToRow : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort @@ -75,7 +75,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -92,7 +92,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.web_returns +- CometSort +- CometExchange +- CometFilter @@ -139,7 +139,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometProject +- CometSortMergeJoin :- CometSort @@ -166,7 +166,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -183,6 +183,6 @@ CometColumnarToRow +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_returns + +- CometNativeScan parquet spark_catalog.default.web_returns Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt index e18882dfff..03ba668f90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt @@ -43,7 +43,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -105,7 +105,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -118,7 +118,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_page : +- CometBroadcastExchange : +- CometHashAggregate : +- CometExchange @@ -128,7 +128,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -141,7 +141,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -185,7 +185,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -247,7 +247,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -260,7 +260,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_page : +- CometBroadcastExchange : +- CometHashAggregate : +- CometExchange @@ -270,7 +270,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -283,7 +283,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -327,7 +327,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -389,7 +389,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -402,7 +402,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- CometBroadcastExchange +- CometHashAggregate +- CometExchange @@ -412,7 +412,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -425,6 +425,6 @@ CometColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_page Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt index c253b8aff8..fa87e0b01d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject : : :- CometSort : : : +- CometExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -53,7 +53,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt index b66eb1ce18..383373ab1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt @@ -79,7 +79,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -87,7 +87,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -112,7 +112,7 @@ CometColumnarToRow : : : : : :- CometSort : : : : : : +- CometExchange : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -123,7 +123,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -131,7 +131,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.web_site : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -218,7 +218,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -226,7 +226,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -251,7 +251,7 @@ CometColumnarToRow : : : : : :- CometSort : : : : : : +- CometExchange : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -262,7 +262,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -270,7 +270,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.web_site : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -357,7 +357,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -365,7 +365,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -390,7 +390,7 @@ CometColumnarToRow : : : : :- CometSort : : : : : +- CometExchange : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -401,7 +401,7 @@ CometColumnarToRow : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -409,7 +409,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_site : +- CometBroadcastExchange : +- CometProject : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index 4b85046c30..bf97aa989e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt @@ -124,7 +124,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -198,7 +198,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -297,7 +297,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -371,7 +371,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt index 367fe17ed9..56a44482b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt @@ -126,7 +126,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -200,7 +200,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -229,7 +229,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -292,7 +292,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -366,7 +366,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -395,7 +395,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -458,7 +458,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -532,7 +532,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -675,7 +675,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -749,7 +749,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -778,7 +778,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -841,7 +841,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -915,7 +915,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -944,7 +944,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1007,7 +1007,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1081,7 +1081,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1224,7 +1224,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1298,7 +1298,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1327,7 +1327,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1390,7 +1390,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1464,7 +1464,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1493,7 +1493,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1556,7 +1556,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1630,7 +1630,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -1773,7 +1773,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -1847,7 +1847,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1876,7 +1876,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -1939,7 +1939,7 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -2013,7 +2013,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2042,7 +2042,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2105,7 +2105,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2179,7 +2179,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2322,7 +2322,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2396,7 +2396,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2425,7 +2425,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2488,7 +2488,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -2562,7 +2562,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2591,7 +2591,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -2654,7 +2654,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -2728,7 +2728,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt index f74b47bf8a..7b433fec06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt @@ -229,6 +229,6 @@ CometColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometNativeScan parquet spark_catalog.default.item Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 4446b1d56b..2916a8248e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt @@ -48,7 +48,7 @@ TakeOrderedAndProject : : +- CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index ff7ad3b219..084971ac28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -54,7 +54,7 @@ CometColumnarToRow : : :- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -86,7 +86,7 @@ CometColumnarToRow : :- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -95,7 +95,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index 6714125d8e..3949c10caa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -83,7 +83,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -106,7 +106,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -198,7 +198,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -221,7 +221,7 @@ TakeOrderedAndProject +- CometProject +- CometBroadcastHashJoin :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometNativeScan parquet spark_catalog.default.store_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index 74efb6d851..88239ea6fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt @@ -52,7 +52,7 @@ CometColumnarToRow : : : :- CometUnion : : : : :- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -61,7 +61,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -75,7 +75,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -113,7 +113,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_site :- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -165,7 +165,7 @@ CometColumnarToRow : : : :- CometUnion : : : : :- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -174,7 +174,7 @@ CometColumnarToRow : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -188,7 +188,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.catalog_page : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate @@ -226,7 +226,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_site +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -278,7 +278,7 @@ CometColumnarToRow : : :- CometUnion : : : :- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -287,7 +287,7 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -301,7 +301,7 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.catalog_page +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -339,6 +339,6 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_site Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt index 8aa248f6ca..f1364542ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt @@ -115,10 +115,10 @@ CometColumnarToRow : : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : : +- CometBroadcastExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : +- CometNativeScan parquet spark_catalog.default.income_band : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -235,10 +235,10 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer_address : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion + : : +- CometNativeScan parquet spark_catalog.default.income_band : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometNativeScan parquet spark_catalog.default.income_band +- CometBroadcastExchange +- CometProject +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt index 7a49cad749..5e78f07bd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt @@ -41,7 +41,7 @@ CometColumnarToRow : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt index 274047acbb..0476be6c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt @@ -63,7 +63,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.customer : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -85,7 +85,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- CometNativeScan parquet spark_catalog.default.web_sales : +- SubqueryBroadcast : +- BroadcastExchange : +- CometColumnarToRow diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt index 33a79ee1f7..5d93f1897b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt @@ -48,7 +48,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -65,7 +65,7 @@ CometColumnarToRow : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- CometNativeScan parquet spark_catalog.default.store_returns : +- CometProject : +- CometSortMergeJoin : :- CometSort @@ -75,7 +75,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -92,7 +92,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.web_returns +- CometSort +- CometExchange +- CometFilter @@ -139,7 +139,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -156,7 +156,7 @@ CometColumnarToRow : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometNativeScan parquet spark_catalog.default.store_returns +- CometProject +- CometSortMergeJoin :- CometSort @@ -166,7 +166,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -183,6 +183,6 @@ CometColumnarToRow +- CometExchange +- CometProject +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_returns + +- CometNativeScan parquet spark_catalog.default.web_returns Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index e18882dfff..03ba668f90 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt @@ -43,7 +43,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -105,7 +105,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -118,7 +118,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_page : +- CometBroadcastExchange : +- CometHashAggregate : +- CometExchange @@ -128,7 +128,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -141,7 +141,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate : +- CometColumnarToRow : +- CometColumnarExchange @@ -185,7 +185,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -247,7 +247,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast : : : : +- BroadcastExchange : : : : +- CometColumnarToRow @@ -260,7 +260,7 @@ CometColumnarToRow : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_page : +- CometBroadcastExchange : +- CometHashAggregate : +- CometExchange @@ -270,7 +270,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -283,7 +283,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate +- CometColumnarToRow +- CometColumnarExchange @@ -327,7 +327,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -389,7 +389,7 @@ CometColumnarToRow : :- CometProject : : +- CometBroadcastHashJoin : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -402,7 +402,7 @@ CometColumnarToRow : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometNativeScan parquet spark_catalog.default.web_page +- CometBroadcastExchange +- CometHashAggregate +- CometExchange @@ -412,7 +412,7 @@ CometColumnarToRow :- CometProject : +- CometBroadcastHashJoin : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- CometNativeScan parquet spark_catalog.default.web_returns : : +- SubqueryBroadcast : : +- BroadcastExchange : : +- CometColumnarToRow @@ -425,6 +425,6 @@ CometColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometNativeScan parquet spark_catalog.default.web_page Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt index c253b8aff8..fa87e0b01d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt @@ -43,7 +43,7 @@ TakeOrderedAndProject : : :- CometSort : : : +- CometExchange : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast : : : +- BroadcastExchange : : : +- CometColumnarToRow @@ -53,7 +53,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : +- CometNativeScan parquet spark_catalog.default.web_returns : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -80,7 +80,7 @@ TakeOrderedAndProject : +- CometExchange : +- CometProject : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns + : +- CometNativeScan parquet spark_catalog.default.catalog_returns +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt index b66eb1ce18..383373ab1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt @@ -79,7 +79,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -87,7 +87,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -112,7 +112,7 @@ CometColumnarToRow : : : : : :- CometSort : : : : : : +- CometExchange : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -123,7 +123,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -131,7 +131,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.web_site : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -218,7 +218,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometProject : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -226,7 +226,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -251,7 +251,7 @@ CometColumnarToRow : : : : : :- CometSort : : : : : : +- CometExchange : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : +- SubqueryBroadcast : : : : : : +- BroadcastExchange : : : : : : +- CometColumnarToRow @@ -262,7 +262,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -270,7 +270,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.web_site : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -357,7 +357,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometProject : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -365,7 +365,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometNativeScan parquet spark_catalog.default.catalog_page : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -390,7 +390,7 @@ CometColumnarToRow : : : : :- CometSort : : : : : +- CometExchange : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast : : : : : +- BroadcastExchange : : : : : +- CometColumnarToRow @@ -401,7 +401,7 @@ CometColumnarToRow : : : : +- CometExchange : : : : +- CometProject : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -409,7 +409,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometNativeScan parquet spark_catalog.default.web_site : +- CometBroadcastExchange : +- CometProject : +- CometFilter From 916c37eafaa669303dcb13a756a0435961b9bc8a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Feb 2026 09:28:37 -0500 Subject: [PATCH 26/38] Try again with canonicalization --- .../spark/sql/comet/CometNativeScanExec.scala | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index a4d93cc114..9aecba0cf6 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -33,9 +33,8 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration import org.apache.spark.util.collection._ -import com.google.common.base.Objects - import org.apache.comet.parquet.CometParquetUtils +import org.apache.comet.serde.OperatorOuterClass import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.operator.CometNativeScan import org.apache.comet.shims.ShimSubqueryBroadcast @@ -178,8 +177,20 @@ case class CometNativeScanExec( } override def doCanonicalize(): CometNativeScanExec = { + // Create minimal canonical nativeOp with just scanId for consistent comparison. + // This matches CometIcebergNativeScanExec's approach of setting originalPlan=null + // and relying on case class field comparison (output, requiredSchema, relation, etc.) + // to distinguish scans. The scanId ensures different tables don't incorrectly match. + val canonicalNativeOp = { + val commonBuilder = OperatorOuterClass.NativeScanCommon.newBuilder() + commonBuilder.setScanId(scanId) + val nativeScanBuilder = OperatorOuterClass.NativeScan.newBuilder() + nativeScanBuilder.setCommon(commonBuilder.build()) + OperatorOuterClass.Operator.newBuilder().setNativeScan(nativeScanBuilder).build() + } + CometNativeScanExec( - nativeOp, + canonicalNativeOp, relation, output.map(QueryPlan.normalizeExpressions(_, output)), requiredSchema, @@ -191,23 +202,16 @@ case class CometNativeScanExec( QueryPlan.normalizePredicates(dataFilters, output), None, disableBucketedScan, - originalPlan.doCanonicalize(), + null, SerializedPlan(None)) } override def stringArgs: Iterator[Any] = Iterator(output) - override def equals(obj: Any): Boolean = { - obj match { - case other: CometNativeScanExec => - this.originalPlan == other.originalPlan && - this.serializedPlanOpt == other.serializedPlanOpt - case _ => - false - } - } - - override def hashCode(): Int = Objects.hashCode(originalPlan, serializedPlanOpt) + // Note: We intentionally use case class default equals/hashCode rather than custom + // implementations. This ensures canonical form comparison considers all fields + // (output, requiredSchema, relation, filters, etc.) - not just originalPlan. + // This prevents incorrect AQE exchange reuse between scans with different projections. override lazy val metrics: Map[String, SQLMetric] = CometMetricNode.nativeScanMetrics(sparkContext) From b2d15406531cf839f658453c1b7c622c806d1fcf Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Feb 2026 09:32:45 -0500 Subject: [PATCH 27/38] Try again with canonicalization --- .../spark/sql/comet/CometNativeScanExec.scala | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 9aecba0cf6..96faeea66e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -34,7 +34,6 @@ import org.apache.spark.util.SerializableConfiguration import org.apache.spark.util.collection._ import org.apache.comet.parquet.CometParquetUtils -import org.apache.comet.serde.OperatorOuterClass import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.operator.CometNativeScan import org.apache.comet.shims.ShimSubqueryBroadcast @@ -177,20 +176,8 @@ case class CometNativeScanExec( } override def doCanonicalize(): CometNativeScanExec = { - // Create minimal canonical nativeOp with just scanId for consistent comparison. - // This matches CometIcebergNativeScanExec's approach of setting originalPlan=null - // and relying on case class field comparison (output, requiredSchema, relation, etc.) - // to distinguish scans. The scanId ensures different tables don't incorrectly match. - val canonicalNativeOp = { - val commonBuilder = OperatorOuterClass.NativeScanCommon.newBuilder() - commonBuilder.setScanId(scanId) - val nativeScanBuilder = OperatorOuterClass.NativeScan.newBuilder() - nativeScanBuilder.setCommon(commonBuilder.build()) - OperatorOuterClass.Operator.newBuilder().setNativeScan(nativeScanBuilder).build() - } - CometNativeScanExec( - canonicalNativeOp, + nativeOp, relation, output.map(QueryPlan.normalizeExpressions(_, output)), requiredSchema, From d142fa36ed638735da82493aa997437a73e0688f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Feb 2026 10:22:59 -0500 Subject: [PATCH 28/38] Update diffs. --- .../q1.native_datafusion/extended.txt | 9 +- .../q10.native_datafusion/extended.txt | 16 +- .../q11.native_datafusion/extended.txt | 14 +- .../q14a.native_datafusion/extended.txt | 121 +--- .../q14b.native_datafusion/extended.txt | 79 +-- .../q17.native_datafusion/extended.txt | 9 +- .../q23a.native_datafusion/extended.txt | 9 +- .../q23b.native_datafusion/extended.txt | 9 +- .../q25.native_datafusion/extended.txt | 9 +- .../q30.native_datafusion/extended.txt | 9 +- .../q31.native_datafusion/extended.txt | 20 +- .../q32.native_datafusion/extended.txt | 9 +- .../q33.native_datafusion/extended.txt | 16 +- .../q35.native_datafusion/extended.txt | 16 +- .../q38.native_datafusion/extended.txt | 16 +- .../q4.native_datafusion/extended.txt | 26 +- .../q49.native_datafusion/extended.txt | 16 +- .../q5.native_datafusion/extended.txt | 37 +- .../q51.native_datafusion/extended.txt | 9 +- .../q54.native_datafusion/extended.txt | 9 +- .../q56.native_datafusion/extended.txt | 16 +- .../q58.native_datafusion/extended.txt | 36 +- .../q60.native_datafusion/extended.txt | 16 +- .../q61.native_datafusion/extended.txt | 9 +- .../q65.native_datafusion/extended.txt | 9 +- .../q66.native_datafusion/extended.txt | 8 +- .../q69.native_datafusion/extended.txt | 16 +- .../q70.native_datafusion/extended.txt | 9 +- .../q71.native_datafusion/extended.txt | 16 +- .../q74.native_datafusion/extended.txt | 14 +- .../q75.native_datafusion/extended.txt | 26 +- .../q77.native_datafusion/extended.txt | 37 +- .../q78.native_datafusion/extended.txt | 14 +- .../q80.native_datafusion/extended.txt | 16 +- .../q81.native_datafusion/extended.txt | 9 +- .../q83.native_datafusion/extended.txt | 34 +- .../q87.native_datafusion/extended.txt | 16 +- .../q92.native_datafusion/extended.txt | 9 +- .../q97.native_datafusion/extended.txt | 9 +- .../q1.native_datafusion/extended.txt | 9 +- .../q10.native_datafusion/extended.txt | 16 +- .../q11.native_datafusion/extended.txt | 14 +- .../q14a.native_datafusion/extended.txt | 121 +--- .../q14b.native_datafusion/extended.txt | 103 +--- .../q17.native_datafusion/extended.txt | 9 +- .../q23a.native_datafusion/extended.txt | 9 +- .../q23b.native_datafusion/extended.txt | 9 +- .../q25.native_datafusion/extended.txt | 9 +- .../q30.native_datafusion/extended.txt | 9 +- .../q31.native_datafusion/extended.txt | 20 +- .../q32.native_datafusion/extended.txt | 9 +- .../q33.native_datafusion/extended.txt | 16 +- .../q35.native_datafusion/extended.txt | 16 +- .../q38.native_datafusion/extended.txt | 16 +- .../q4.native_datafusion/extended.txt | 26 +- .../q49.native_datafusion/extended.txt | 16 +- .../q5.native_datafusion/extended.txt | 37 +- .../q51.native_datafusion/extended.txt | 9 +- .../q54.native_datafusion/extended.txt | 45 +- .../q56.native_datafusion/extended.txt | 16 +- .../q58.native_datafusion/extended.txt | 52 +- .../q6.native_datafusion/extended.txt | 20 +- .../q60.native_datafusion/extended.txt | 16 +- .../q61.native_datafusion/extended.txt | 9 +- .../q65.native_datafusion/extended.txt | 9 +- .../q66.native_datafusion/extended.txt | 8 +- .../q69.native_datafusion/extended.txt | 16 +- .../q70.native_datafusion/extended.txt | 9 +- .../q71.native_datafusion/extended.txt | 16 +- .../q74.native_datafusion/extended.txt | 14 +- .../q75.native_datafusion/extended.txt | 26 +- .../q77.native_datafusion/extended.txt | 37 +- .../q78.native_datafusion/extended.txt | 14 +- .../q80.native_datafusion/extended.txt | 16 +- .../q81.native_datafusion/extended.txt | 9 +- .../q83.ansi.native_datafusion/extended.txt | 34 +- .../q87.native_datafusion/extended.txt | 16 +- .../q92.native_datafusion/extended.txt | 9 +- .../q97.native_datafusion/extended.txt | 9 +- .../q1.native_datafusion/extended.txt | 9 +- .../q10.native_datafusion/extended.txt | 16 +- .../q11.native_datafusion/extended.txt | 14 +- .../q14a.native_datafusion/extended.txt | 121 +--- .../q14b.native_datafusion/extended.txt | 79 +-- .../q17.native_datafusion/extended.txt | 9 +- .../q23a.native_datafusion/extended.txt | 9 +- .../q23b.native_datafusion/extended.txt | 9 +- .../q25.native_datafusion/extended.txt | 9 +- .../q30.native_datafusion/extended.txt | 9 +- .../q31.native_datafusion/extended.txt | 20 +- .../q32.native_datafusion/extended.txt | 9 +- .../q33.native_datafusion/extended.txt | 16 +- .../q35.native_datafusion/extended.txt | 16 +- .../q38.native_datafusion/extended.txt | 16 +- .../q4.native_datafusion/extended.txt | 26 +- .../q49.native_datafusion/extended.txt | 16 +- .../q5.native_datafusion/extended.txt | 37 +- .../q51.native_datafusion/extended.txt | 9 +- .../q54.native_datafusion/extended.txt | 9 +- .../q56.native_datafusion/extended.txt | 16 +- .../q58.native_datafusion/extended.txt | 25 +- .../q60.native_datafusion/extended.txt | 16 +- .../q61.native_datafusion/extended.txt | 9 +- .../q65.native_datafusion/extended.txt | 9 +- .../q66.native_datafusion/extended.txt | 8 +- .../q69.native_datafusion/extended.txt | 16 +- .../q70.native_datafusion/extended.txt | 9 +- .../q71.native_datafusion/extended.txt | 16 +- .../q74.native_datafusion/extended.txt | 14 +- .../q75.native_datafusion/extended.txt | 26 +- .../q77.native_datafusion/extended.txt | 37 +- .../q78.native_datafusion/extended.txt | 14 +- .../q80.native_datafusion/extended.txt | 16 +- .../q81.native_datafusion/extended.txt | 9 +- .../q83.native_datafusion/extended.txt | 18 +- .../q87.native_datafusion/extended.txt | 16 +- .../q92.native_datafusion/extended.txt | 9 +- .../q97.native_datafusion/extended.txt | 9 +- .../q10a.native_datafusion/extended.txt | 16 +- .../q11.native_datafusion/extended.txt | 14 +- .../q14.native_datafusion/extended.txt | 79 +-- .../q14a.native_datafusion/extended.txt | 562 +++--------------- .../q18a.native_datafusion/extended.txt | 30 +- .../q27a.native_datafusion/extended.txt | 16 +- .../q35.native_datafusion/extended.txt | 16 +- .../q35a.native_datafusion/extended.txt | 16 +- .../q49.native_datafusion/extended.txt | 16 +- .../q51a.native_datafusion/extended.txt | 30 +- .../q5a.native_datafusion/extended.txt | 107 +--- .../q70a.native_datafusion/extended.txt | 23 +- .../q74.native_datafusion/extended.txt | 14 +- .../q75.native_datafusion/extended.txt | 26 +- .../q77a.native_datafusion/extended.txt | 107 +--- .../q78.native_datafusion/extended.txt | 14 +- .../q80a.native_datafusion/extended.txt | 44 +- .../q10a.native_datafusion/extended.txt | 16 +- .../q11.native_datafusion/extended.txt | 14 +- .../q14.native_datafusion/extended.txt | 103 +--- .../q14a.native_datafusion/extended.txt | 562 +++--------------- .../q18a.native_datafusion/extended.txt | 30 +- .../q27a.native_datafusion/extended.txt | 16 +- .../q35.native_datafusion/extended.txt | 16 +- .../q35a.native_datafusion/extended.txt | 16 +- .../q49.native_datafusion/extended.txt | 16 +- .../q51a.native_datafusion/extended.txt | 30 +- .../q5a.native_datafusion/extended.txt | 107 +--- .../q6.native_datafusion/extended.txt | 20 +- .../q70a.native_datafusion/extended.txt | 23 +- .../q74.native_datafusion/extended.txt | 14 +- .../q75.native_datafusion/extended.txt | 26 +- .../q77a.native_datafusion/extended.txt | 107 +--- .../q78.native_datafusion/extended.txt | 14 +- .../q80a.native_datafusion/extended.txt | 44 +- .../q10a.native_datafusion/extended.txt | 16 +- .../q11.native_datafusion/extended.txt | 14 +- .../q14.native_datafusion/extended.txt | 79 +-- .../q14a.native_datafusion/extended.txt | 562 +++--------------- .../q18a.native_datafusion/extended.txt | 30 +- .../q27a.native_datafusion/extended.txt | 16 +- .../q35.native_datafusion/extended.txt | 16 +- .../q35a.native_datafusion/extended.txt | 16 +- .../q49.native_datafusion/extended.txt | 16 +- .../q51a.native_datafusion/extended.txt | 30 +- .../q5a.native_datafusion/extended.txt | 107 +--- .../q70a.native_datafusion/extended.txt | 23 +- .../q74.native_datafusion/extended.txt | 14 +- .../q75.native_datafusion/extended.txt | 26 +- .../q77a.native_datafusion/extended.txt | 107 +--- .../q78.native_datafusion/extended.txt | 14 +- .../q80a.native_datafusion/extended.txt | 44 +- 170 files changed, 951 insertions(+), 4883 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt index 901ca3692b..7293d262f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt @@ -36,12 +36,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -55,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt index 6e6f311b7d..e69fb2ee81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt @@ -65,11 +65,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,13 +83,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt index ab185e742b..2fefb0e3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt @@ -16,12 +16,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -29,12 +24,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,12 +32,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,12 +86,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +105,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -175,12 +150,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +169,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -229,12 +194,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -268,12 +228,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -292,12 +247,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -342,12 +292,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -366,12 +311,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +336,7 @@ CometColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -435,12 +370,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +389,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -509,12 +434,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +453,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -551,4 +466,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt index bf97aa989e..16aa5801f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt @@ -11,12 +11,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +19,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +27,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,12 +86,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +105,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -175,12 +150,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +169,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -274,12 +239,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -298,12 +258,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -348,12 +303,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -372,12 +322,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -391,4 +336,4 @@ CometColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt index c2f112b9bf..b168abc169 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt @@ -37,12 +37,7 @@ CometColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt index 6151d372dc..cb936c9470 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt @@ -97,12 +97,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -148,4 +143,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt index cda2b25dfe..750bc39460 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt @@ -127,12 +127,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,4 +195,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt index c2f112b9bf..b168abc169 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt @@ -37,12 +37,7 @@ CometColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt index 49560ff5d3..d7fec54ac1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt index 680121f013..0e4c97d938 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt @@ -81,11 +81,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,11 +98,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,11 +115,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,4 +123,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt index 3ef52bc501..7a02843f94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt @@ -29,12 +29,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt index 89aa813e43..3755a89ee0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt @@ -48,12 +48,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,12 +76,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +94,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt index 1a323c6c34..3a72b7cb9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt @@ -38,12 +38,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +57,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt index 2a5b362a3e..2b3c9ee512 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt @@ -68,11 +68,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +86,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -113,11 +105,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,13 +123,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt index 084971ac28..a7caa82ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt @@ -55,12 +55,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +82,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt index 1472b14332..1abbe590ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt @@ -25,12 +25,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -50,21 +45,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,22 +69,12 @@ CometColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -112,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt index d60ef8792f..3faa5d6b77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt @@ -44,15 +44,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt index f5e651b035..b5161dbc09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt @@ -36,12 +36,7 @@ CometColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -93,4 +88,4 @@ CometColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 78 out of 88 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt index 499942b26c..9b0203f98a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +78,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt index f86ed28206..9110f8f632 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt @@ -59,22 +59,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,22 +89,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -139,4 +109,4 @@ CometColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 118 out of 130 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 96 out of 104 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt index 499942b26c..9b0203f98a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +78,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt index a8cafce32e..c54bef9b6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt @@ -64,12 +64,7 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -90,4 +85,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt index 402cc842ca..7265aa9514 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt @@ -43,15 +43,10 @@ CometColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt index 209f7a9f0f..7112fbed2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt @@ -50,11 +50,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -71,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt index 8aa58425a7..85ed0baa1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt @@ -33,12 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt index f0b51ba168..a0ad8fbd3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt @@ -48,12 +48,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -63,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 57 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt index 5d11aad358..4dfcc2f8ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -45,12 +40,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -60,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt index 5d93f1897b..f07b682231 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt @@ -49,11 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +72,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +132,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +155,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt index e4912d3273..c3675de988 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt @@ -40,12 +40,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,12 +58,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,12 +70,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -102,12 +87,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -125,12 +105,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -139,4 +114,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt index fa87e0b01d..5b69269c61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt @@ -44,11 +44,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +67,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt index bb1998687e..cef9fe5220 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt @@ -66,12 +66,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -110,12 +105,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -138,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt index aab91b60e9..f089c495ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt index d10feb6fb0..aeecb4336a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt @@ -55,21 +55,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -97,21 +83,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -130,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt index 5b119a3926..dde12be29e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt @@ -40,12 +40,7 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -65,12 +60,7 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -80,4 +70,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt index 5695ae40ec..9e5e99015a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt @@ -29,12 +29,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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-spark3_5/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt index 9fe719d3c7..331f00fe36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt @@ -28,15 +28,10 @@ CometColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt index 901ca3692b..7293d262f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt @@ -36,12 +36,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -55,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt index 6e6f311b7d..e69fb2ee81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt @@ -65,11 +65,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,13 +83,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt index ab185e742b..2fefb0e3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt @@ -16,12 +16,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -29,12 +24,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,12 +32,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,12 +86,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +105,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -175,12 +150,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +169,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -229,12 +194,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -268,12 +228,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -292,12 +247,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -342,12 +292,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -366,12 +311,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +336,7 @@ CometColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -435,12 +370,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +389,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -509,12 +434,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +453,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -551,4 +466,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt index 2b89b148f1..7260efac0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt @@ -11,12 +11,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +19,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +27,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,11 +47,7 @@ CometColumnarToRow : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery : : : : +- CometColumnarToRow @@ -106,12 +87,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -130,12 +106,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -180,12 +151,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -204,12 +170,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -222,11 +183,7 @@ CometColumnarToRow : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter : +- ReusedSubquery @@ -245,11 +202,7 @@ CometColumnarToRow : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- Subquery : : : +- CometColumnarToRow @@ -289,12 +242,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -313,12 +261,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -363,12 +306,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -387,12 +325,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -405,10 +338,6 @@ CometColumnarToRow +- CometFilter : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- ReusedSubquery -Comet accelerated 343 out of 387 eligible operators (88%). Final plan contains 25 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt index c2f112b9bf..b168abc169 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt @@ -37,12 +37,7 @@ CometColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt index 6151d372dc..cb936c9470 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt @@ -97,12 +97,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -148,4 +143,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt index cda2b25dfe..750bc39460 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt @@ -127,12 +127,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,4 +195,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt index c2f112b9bf..b168abc169 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt @@ -37,12 +37,7 @@ CometColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt index 49560ff5d3..d7fec54ac1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt index 680121f013..0e4c97d938 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt @@ -81,11 +81,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,11 +98,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,11 +115,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,4 +123,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt index 3ef52bc501..7a02843f94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt @@ -29,12 +29,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt index 89aa813e43..3755a89ee0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt @@ -48,12 +48,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,12 +76,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +94,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt index 1a323c6c34..3a72b7cb9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt @@ -38,12 +38,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +57,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt index 2a5b362a3e..2b3c9ee512 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt @@ -68,11 +68,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +86,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -113,11 +105,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,13 +123,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt index 084971ac28..a7caa82ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt @@ -55,12 +55,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +82,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt index 1472b14332..1abbe590ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt @@ -25,12 +25,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -50,21 +45,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,22 +69,12 @@ CometColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -112,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt index d60ef8792f..3faa5d6b77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt @@ -44,15 +44,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt index 9f071b0261..798f7eed45 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt @@ -36,12 +36,7 @@ CometColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -61,22 +56,8 @@ CometColumnarToRow : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : :- ReusedSubquery + : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : :- Subquery : : : : +- CometColumnarToRow @@ -108,21 +89,7 @@ CometColumnarToRow : :- ReusedSubquery : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim - :- Subquery - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + :- ReusedSubquery + +- ReusedSubquery -Comet accelerated 102 out of 116 eligible operators (87%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt index 499942b26c..9b0203f98a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +78,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt index ceb98a1e98..6a006afae8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt @@ -24,11 +24,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery : : : : +- CometColumnarToRow @@ -65,27 +61,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -116,27 +92,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -157,4 +113,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 127 out of 145 eligible operators (87%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt index 75418138a1..bd21dfad52 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt @@ -26,14 +26,7 @@ CometColumnarToRow : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- Subquery : : +- CometColumnarToRow @@ -48,14 +41,7 @@ CometColumnarToRow : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometBroadcastHashJoin @@ -70,4 +56,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 60 out of 66 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt index 499942b26c..9b0203f98a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +78,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt index a8cafce32e..c54bef9b6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt @@ -64,12 +64,7 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -90,4 +85,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt index 402cc842ca..7265aa9514 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt @@ -43,15 +43,10 @@ CometColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt index 209f7a9f0f..7112fbed2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt @@ -50,11 +50,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -71,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt index 8aa58425a7..85ed0baa1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt @@ -33,12 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt index f0b51ba168..a0ad8fbd3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt @@ -48,12 +48,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -63,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 57 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt index 5d11aad358..4dfcc2f8ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -45,12 +40,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -60,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt index 5d93f1897b..f07b682231 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt @@ -49,11 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +72,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +132,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +155,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt index e4912d3273..c3675de988 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt @@ -40,12 +40,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,12 +58,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,12 +70,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -102,12 +87,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -125,12 +105,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -139,4 +114,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt index fa87e0b01d..5b69269c61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt @@ -44,11 +44,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +67,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt index bb1998687e..cef9fe5220 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt @@ -66,12 +66,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -110,12 +105,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -138,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt index aab91b60e9..f089c495ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt index d10feb6fb0..aeecb4336a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt @@ -55,21 +55,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -97,21 +83,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -130,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt index 5b119a3926..dde12be29e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt @@ -40,12 +40,7 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -65,12 +60,7 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -80,4 +70,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 66 eligible operators (83%). 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/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt index 5695ae40ec..9e5e99015a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt @@ -29,12 +29,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt index 9fe719d3c7..331f00fe36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt @@ -28,15 +28,10 @@ CometColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt index 901ca3692b..7293d262f9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt @@ -36,12 +36,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -55,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt index 6e6f311b7d..e69fb2ee81 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt @@ -65,11 +65,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,13 +83,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index ab185e742b..2fefb0e3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt @@ -16,12 +16,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -29,12 +24,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,12 +32,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,12 +86,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +105,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -175,12 +150,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +169,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -229,12 +194,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -268,12 +228,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -292,12 +247,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -342,12 +292,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -366,12 +311,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +336,7 @@ CometColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -435,12 +370,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +389,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -509,12 +434,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +453,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -551,4 +466,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt index bf97aa989e..16aa5801f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt @@ -11,12 +11,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +19,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +27,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,12 +86,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +105,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -175,12 +150,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +169,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -274,12 +239,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -298,12 +258,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -348,12 +303,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -372,12 +322,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -391,4 +336,4 @@ CometColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt index c2f112b9bf..b168abc169 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt @@ -37,12 +37,7 @@ CometColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt index 6151d372dc..cb936c9470 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt @@ -97,12 +97,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -148,4 +143,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt index cda2b25dfe..750bc39460 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt @@ -127,12 +127,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,4 +195,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index c2f112b9bf..b168abc169 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt @@ -37,12 +37,7 @@ CometColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt index 49560ff5d3..d7fec54ac1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index 680121f013..0e4c97d938 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt @@ -81,11 +81,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,11 +98,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,11 +115,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,4 +123,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt index 3ef52bc501..7a02843f94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt @@ -29,12 +29,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt index 89aa813e43..3755a89ee0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt @@ -48,12 +48,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,12 +76,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +94,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt index 1a323c6c34..3a72b7cb9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt @@ -38,12 +38,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +57,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index 2a5b362a3e..2b3c9ee512 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt @@ -68,11 +68,7 @@ CometColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +86,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -113,11 +105,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,13 +123,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index 084971ac28..a7caa82ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -55,12 +55,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +82,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt index 1472b14332..1abbe590ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt @@ -25,12 +25,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -50,21 +45,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,22 +69,12 @@ CometColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -112,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index d60ef8792f..3faa5d6b77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -44,15 +44,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt index f5e651b035..b5161dbc09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt @@ -36,12 +36,7 @@ CometColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -93,4 +88,4 @@ CometColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 78 out of 88 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt index 499942b26c..9b0203f98a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +78,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt index 6a89786dc2..de4439657e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt @@ -69,11 +69,7 @@ CometColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject @@ -100,22 +96,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -131,4 +112,4 @@ CometColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 112 out of 124 eligible operators (90%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 108 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index 499942b26c..9b0203f98a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +78,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index a8cafce32e..c54bef9b6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt @@ -64,12 +64,7 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -90,4 +85,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt index 402cc842ca..7265aa9514 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt @@ -43,15 +43,10 @@ CometColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index 209f7a9f0f..7112fbed2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt @@ -50,11 +50,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -71,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index 8aa58425a7..85ed0baa1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt @@ -33,12 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt index 1d85f12ca6..73d2c0bda8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt @@ -47,12 +47,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -62,4 +57,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 56 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 52 eligible operators (65%). 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/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt index 5d11aad358..4dfcc2f8ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -45,12 +40,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -60,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt index 5d93f1897b..f07b682231 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt @@ -49,11 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +72,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +132,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +155,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt index e4912d3273..c3675de988 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt @@ -40,12 +40,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,12 +58,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,12 +70,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -102,12 +87,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -125,12 +105,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -139,4 +114,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt index fa87e0b01d..5b69269c61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt @@ -44,11 +44,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +67,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt index bb1998687e..cef9fe5220 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt @@ -66,12 +66,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -110,12 +105,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -138,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt index aab91b60e9..f089c495ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index d10feb6fb0..7e841ac579 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt @@ -97,21 +97,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -130,4 +116,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 109 out of 114 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/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index 5b119a3926..dde12be29e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt @@ -40,12 +40,7 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -65,12 +60,7 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -80,4 +70,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 66 eligible operators (83%). 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/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt index 5695ae40ec..9e5e99015a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt @@ -29,12 +29,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt index 9fe719d3c7..331f00fe36 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt @@ -28,15 +28,10 @@ CometColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt index 818dd70bdf..f6f71ccf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt index bf97aa989e..16aa5801f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt @@ -11,12 +11,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +19,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +27,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,12 +86,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +105,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -175,12 +150,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +169,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -274,12 +239,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -298,12 +258,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -348,12 +303,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -372,12 +322,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -391,4 +336,4 @@ CometColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt index 56a44482b4..a48a4f02cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt @@ -18,12 +18,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -103,12 +93,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -127,12 +112,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -177,12 +157,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -201,12 +176,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -230,12 +200,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -269,12 +234,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -293,12 +253,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -343,12 +298,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -367,12 +317,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +341,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -435,12 +375,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +394,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -509,12 +439,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +458,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -567,12 +487,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -593,12 +508,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -652,12 +562,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -676,12 +581,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -726,12 +626,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -750,12 +645,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -779,12 +669,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -818,12 +703,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -842,12 +722,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -892,12 +767,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -916,12 +786,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -945,12 +810,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -984,12 +844,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1008,12 +863,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1058,12 +908,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1082,12 +927,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1116,12 +956,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1142,12 +977,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1201,12 +1031,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1225,12 +1050,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1275,12 +1095,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1299,12 +1114,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1328,12 +1138,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1367,12 +1172,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1391,12 +1191,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1441,12 +1236,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1465,12 +1255,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1494,12 +1279,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1533,12 +1313,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1557,12 +1332,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1607,12 +1377,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1631,12 +1396,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1665,12 +1425,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1691,12 +1446,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1750,12 +1500,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1774,12 +1519,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1824,12 +1564,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1848,12 +1583,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1877,12 +1607,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1916,12 +1641,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1940,12 +1660,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1990,12 +1705,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,12 +1724,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2043,12 +1748,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2082,12 +1782,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2106,12 +1801,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2156,12 +1846,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2180,12 +1865,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2214,12 +1894,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2240,12 +1915,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2299,12 +1969,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2323,12 +1988,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2373,12 +2033,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2397,12 +2052,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2426,12 +2076,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2465,12 +2110,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2489,12 +2129,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2539,12 +2174,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2563,12 +2193,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2592,12 +2217,7 @@ CometColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2631,12 +2251,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2655,12 +2270,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2705,12 +2315,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2729,12 +2334,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2747,4 +2347,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt index 7b433fec06..0c126ae714 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt @@ -64,12 +64,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -110,12 +105,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -156,12 +146,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -202,12 +187,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -231,4 +211,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt index 8cc2a333f1..68e31b1eeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,12 +79,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -106,4 +96,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt index 818dd70bdf..f6f71ccf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt index 084971ac28..a7caa82ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt @@ -55,12 +55,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +82,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt index 3949c10caa..d76150bc44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt @@ -84,12 +84,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,12 +102,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -199,12 +189,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -222,15 +207,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt index 88239ea6fe..1621fcebdd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt @@ -28,12 +28,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -53,21 +48,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -87,22 +72,12 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -141,12 +116,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -166,21 +136,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,22 +160,12 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -254,12 +204,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -279,21 +224,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -313,22 +248,12 @@ CometColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -341,4 +266,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt index 1de2522a2c..34f65c79d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt @@ -51,12 +51,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -112,12 +107,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -173,12 +163,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -188,4 +173,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 111 out of 168 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt index 5d93f1897b..f07b682231 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt @@ -49,11 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +72,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +132,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +155,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt index 03ba668f90..df51f4120c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,12 +62,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -84,12 +74,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -106,12 +91,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -129,12 +109,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -186,12 +161,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -209,12 +179,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -226,12 +191,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -248,12 +208,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -271,12 +226,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -328,12 +278,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -351,12 +296,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -368,12 +308,7 @@ CometColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -390,12 +325,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -413,12 +343,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -427,4 +352,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt index fa87e0b01d..5b69269c61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt @@ -44,11 +44,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +67,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt index 383373ab1b..f56f66b4dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt @@ -69,12 +69,7 @@ CometColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -113,12 +108,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -208,12 +198,7 @@ CometColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -252,12 +237,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -347,12 +327,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -391,12 +366,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -419,4 +389,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt index 818dd70bdf..f6f71ccf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt index 2b89b148f1..7260efac0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt @@ -11,12 +11,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +19,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +27,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,11 +47,7 @@ CometColumnarToRow : : : : +- CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery : : : : +- CometColumnarToRow @@ -106,12 +87,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -130,12 +106,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -180,12 +151,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -204,12 +170,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -222,11 +183,7 @@ CometColumnarToRow : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter : +- ReusedSubquery @@ -245,11 +202,7 @@ CometColumnarToRow : : : +- CometColumnarToRow : : : +- CometProject : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- Subquery : : : +- CometColumnarToRow @@ -289,12 +242,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -313,12 +261,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -363,12 +306,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -387,12 +325,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -405,10 +338,6 @@ CometColumnarToRow +- CometFilter : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- ReusedSubquery -Comet accelerated 343 out of 387 eligible operators (88%). Final plan contains 25 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt index 56a44482b4..a48a4f02cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt @@ -18,12 +18,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -103,12 +93,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -127,12 +112,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -177,12 +157,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -201,12 +176,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -230,12 +200,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -269,12 +234,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -293,12 +253,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -343,12 +298,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -367,12 +317,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +341,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -435,12 +375,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +394,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -509,12 +439,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +458,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -567,12 +487,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -593,12 +508,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -652,12 +562,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -676,12 +581,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -726,12 +626,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -750,12 +645,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -779,12 +669,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -818,12 +703,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -842,12 +722,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -892,12 +767,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -916,12 +786,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -945,12 +810,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -984,12 +844,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1008,12 +863,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1058,12 +908,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1082,12 +927,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1116,12 +956,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1142,12 +977,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1201,12 +1031,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1225,12 +1050,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1275,12 +1095,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1299,12 +1114,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1328,12 +1138,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1367,12 +1172,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1391,12 +1191,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1441,12 +1236,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1465,12 +1255,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1494,12 +1279,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1533,12 +1313,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1557,12 +1332,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1607,12 +1377,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1631,12 +1396,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1665,12 +1425,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1691,12 +1446,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1750,12 +1500,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1774,12 +1519,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1824,12 +1564,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1848,12 +1583,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1877,12 +1607,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1916,12 +1641,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1940,12 +1660,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1990,12 +1705,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,12 +1724,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2043,12 +1748,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2082,12 +1782,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2106,12 +1801,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2156,12 +1846,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2180,12 +1865,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2214,12 +1894,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2240,12 +1915,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2299,12 +1969,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2323,12 +1988,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2373,12 +2033,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2397,12 +2052,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2426,12 +2076,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2465,12 +2110,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2489,12 +2129,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2539,12 +2174,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2563,12 +2193,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2592,12 +2217,7 @@ CometColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2631,12 +2251,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2655,12 +2270,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2705,12 +2315,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2729,12 +2334,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2747,4 +2347,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt index 7b433fec06..0c126ae714 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt @@ -64,12 +64,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -110,12 +105,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -156,12 +146,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -202,12 +187,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -231,4 +211,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt index 8cc2a333f1..68e31b1eeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,12 +79,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -106,4 +96,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt index 818dd70bdf..f6f71ccf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt index 084971ac28..a7caa82ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt @@ -55,12 +55,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +82,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt index 3949c10caa..d76150bc44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt @@ -84,12 +84,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,12 +102,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -199,12 +189,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -222,15 +207,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt index 88239ea6fe..1621fcebdd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt @@ -28,12 +28,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -53,21 +48,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -87,22 +72,12 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -141,12 +116,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -166,21 +136,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,22 +160,12 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -254,12 +204,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -279,21 +224,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -313,22 +248,12 @@ CometColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -341,4 +266,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt index 75418138a1..bd21dfad52 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt @@ -26,14 +26,7 @@ CometColumnarToRow : : +- CometColumnarToRow : : +- CometProject : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- Subquery : : +- CometColumnarToRow @@ -48,14 +41,7 @@ CometColumnarToRow : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometBroadcastHashJoin @@ -70,4 +56,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 60 out of 66 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt index 1de2522a2c..34f65c79d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt @@ -51,12 +51,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -112,12 +107,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -173,12 +163,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -188,4 +173,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 111 out of 168 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt index 5d93f1897b..f07b682231 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt @@ -49,11 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +72,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +132,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +155,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt index 03ba668f90..df51f4120c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,12 +62,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -84,12 +74,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -106,12 +91,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -129,12 +109,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -186,12 +161,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -209,12 +179,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -226,12 +191,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -248,12 +208,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -271,12 +226,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -328,12 +278,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -351,12 +296,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -368,12 +308,7 @@ CometColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -390,12 +325,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -413,12 +343,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -427,4 +352,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt index fa87e0b01d..5b69269c61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt @@ -44,11 +44,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +67,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt index 383373ab1b..f56f66b4dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt @@ -69,12 +69,7 @@ CometColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -113,12 +108,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -208,12 +198,7 @@ CometColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -252,12 +237,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -347,12 +327,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -391,12 +366,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -419,4 +389,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt index 818dd70bdf..f6f71ccf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index bf97aa989e..16aa5801f1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt @@ -11,12 +11,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +19,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +27,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,12 +86,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +105,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -175,12 +150,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +169,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -274,12 +239,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -298,12 +258,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -348,12 +303,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -372,12 +322,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -391,4 +336,4 @@ CometColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt index 56a44482b4..a48a4f02cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt @@ -18,12 +18,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -103,12 +93,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -127,12 +112,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -177,12 +157,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -201,12 +176,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -230,12 +200,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -269,12 +234,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -293,12 +253,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -343,12 +298,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -367,12 +317,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +341,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -435,12 +375,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +394,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -509,12 +439,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +458,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -567,12 +487,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -593,12 +508,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -652,12 +562,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -676,12 +581,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -726,12 +626,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -750,12 +645,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -779,12 +669,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -818,12 +703,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -842,12 +722,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -892,12 +767,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -916,12 +786,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -945,12 +810,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -984,12 +844,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1008,12 +863,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1058,12 +908,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1082,12 +927,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1116,12 +956,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1142,12 +977,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1201,12 +1031,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1225,12 +1050,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1275,12 +1095,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1299,12 +1114,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1328,12 +1138,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1367,12 +1172,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1391,12 +1191,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1441,12 +1236,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1465,12 +1255,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1494,12 +1279,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1533,12 +1313,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1557,12 +1332,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1607,12 +1377,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1631,12 +1396,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1665,12 +1425,7 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1691,12 +1446,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1750,12 +1500,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1774,12 +1519,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1824,12 +1564,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1848,12 +1583,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1877,12 +1607,7 @@ CometColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1916,12 +1641,7 @@ CometColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1940,12 +1660,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1990,12 +1705,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,12 +1724,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2043,12 +1748,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2082,12 +1782,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2106,12 +1801,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2156,12 +1846,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2180,12 +1865,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2214,12 +1894,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2240,12 +1915,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2299,12 +1969,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2323,12 +1988,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2373,12 +2033,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2397,12 +2052,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2426,12 +2076,7 @@ CometColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2465,12 +2110,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2489,12 +2129,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2539,12 +2174,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2563,12 +2193,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2592,12 +2217,7 @@ CometColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2631,12 +2251,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2655,12 +2270,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2705,12 +2315,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2729,12 +2334,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2747,4 +2347,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt index 7b433fec06..0c126ae714 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt @@ -64,12 +64,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -110,12 +105,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -156,12 +146,7 @@ CometColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -202,12 +187,7 @@ CometColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -231,4 +211,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt index 8cc2a333f1..68e31b1eeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt @@ -49,12 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,12 +79,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -106,4 +96,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 2916a8248e..4472ddfbbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt @@ -34,12 +34,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +44,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt index 818dd70bdf..f6f71ccf92 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt @@ -31,12 +31,7 @@ CometColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +39,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index 084971ac28..a7caa82ba5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -55,12 +55,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +82,7 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index 3949c10caa..d76150bc44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -84,12 +84,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,12 +102,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -199,12 +189,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -222,15 +207,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index 88239ea6fe..1621fcebdd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt @@ -28,12 +28,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -53,21 +48,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -87,22 +72,12 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -141,12 +116,7 @@ CometColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -166,21 +136,11 @@ CometColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,22 +160,12 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -254,12 +204,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -279,21 +224,11 @@ CometColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -313,22 +248,12 @@ CometColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -341,4 +266,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt index 527b3ae2d3..27ffd38df9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt @@ -50,12 +50,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -110,12 +105,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -170,12 +160,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -185,4 +170,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 111 out of 165 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 102 out of 153 eligible operators (66%). 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/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt index 0476be6c98..e0ed00033c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt @@ -64,11 +64,7 @@ CometColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +82,9 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt index 5d93f1897b..f07b682231 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt @@ -49,11 +49,7 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +72,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +132,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +155,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index 03ba668f90..df51f4120c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt @@ -44,12 +44,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,12 +62,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -84,12 +74,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -106,12 +91,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -129,12 +109,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -186,12 +161,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -209,12 +179,7 @@ CometColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -226,12 +191,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -248,12 +208,7 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -271,12 +226,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -328,12 +278,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -351,12 +296,7 @@ CometColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -368,12 +308,7 @@ CometColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -390,12 +325,7 @@ CometColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -413,12 +343,7 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -427,4 +352,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt index fa87e0b01d..5b69269c61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt @@ -44,11 +44,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +67,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 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/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt index 383373ab1b..f56f66b4dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt @@ -69,12 +69,7 @@ CometColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -113,12 +108,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -208,12 +198,7 @@ CometColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -252,12 +237,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -347,12 +327,7 @@ CometColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -391,12 +366,7 @@ CometColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -419,4 +389,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file From 288a248ce95c74db3e423c59974cdb7e8b660fb6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Feb 2026 10:27:12 -0500 Subject: [PATCH 29/38] Attempt to fix "SPARK-30291: AQE should catch the exceptions when doing materialize" --- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 3 ++- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 3 ++- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 710f0ebfc5..955f3f4eb7 100644 --- a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} +import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions @@ -125,7 +126,7 @@ trait ShimCometScanExec { .groupBy { f => BucketingUtils .getBucketId(f.toPath.getName) - .getOrElse(throw new IllegalStateException(s"Invalid bucket file: ${f.toPath}")) + .getOrElse(throw QueryExecutionErrors.invalidBucketFile(f.toPath.toString)) } // Apply bucket pruning diff --git a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 695d73707e..e066b672c2 100644 --- a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} +import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions @@ -127,7 +128,7 @@ trait ShimCometScanExec { .groupBy { f => BucketingUtils .getBucketId(f.toPath.getName) - .getOrElse(throw new IllegalStateException(s"Invalid bucket file: ${f.toPath}")) + .getOrElse(throw QueryExecutionErrors.invalidBucketFile(f.toPath.toString)) } // Apply bucket pruning diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 1c443e5d13..e98acfff6a 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, FileSourceConstantMetadataAttribute, Literal, Predicate} +import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil, ScalarSubquery, ScanFileListing} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions @@ -128,7 +129,7 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { .groupBy { f => BucketingUtils .getBucketId(f.toPath.getName) - .getOrElse(throw new IllegalStateException(s"Invalid bucket file: ${f.toPath}")) + .getOrElse(throw QueryExecutionErrors.invalidBucketFile(f.toPath.toString)) } // Apply bucket pruning From 10f9e42ad1cc2d8d5b69ec028eb15f3b38b4137a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Feb 2026 11:22:01 -0500 Subject: [PATCH 30/38] fix "DPP with native_datafusion scan - join with dynamic partition pruning" --- .../spark/sql/comet/CometNativeScanExec.scala | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 96faeea66e..0c33d6e421 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -22,6 +22,7 @@ package org.apache.spark.sql.comet import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.comet.shims.ShimCometScanExec @@ -85,7 +86,11 @@ case class CometNativeScanExec( * Prepare DPP subquery plans. Called by Spark's prepare() before doExecuteColumnar(). */ override protected def doPrepare(): Unit = { - partitionFilters.foreach { + // Use originalPlan.partitionFilters (not partitionFilters) because AQE's + // PlanDynamicPruningFilters may transform InSubqueryExec → Literal.TrueLiteral + // via makeCopy, but originalPlan is not in the active plan tree so it retains + // the original InSubqueryExec needed for DPP preparation. + originalPlan.partitionFilters.foreach { case DynamicPruningExpression(e: InSubqueryExec) => e.plan.prepare() case _ => @@ -97,8 +102,12 @@ case class CometNativeScanExec( * Lazy partition serialization - deferred until execution time for DPP support. */ @transient private lazy val serializedPartitionData: (Array[Byte], Array[Array[Byte]]) = { - // Wait for DPP subqueries to resolve before accessing partitions - partitionFilters.foreach { + // Wait for DPP subqueries to resolve before accessing partitions. + // Use originalPlan.partitionFilters (not partitionFilters) because AQE's + // PlanDynamicPruningFilters may transform InSubqueryExec → Literal.TrueLiteral + // via makeCopy, but originalPlan is not in the active plan tree so it retains + // the original InSubqueryExec needed for DPP resolution. + originalPlan.partitionFilters.foreach { case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => e.plan match { case sab: SubqueryAdaptiveBroadcastExec => @@ -118,16 +127,22 @@ case class CometNativeScanExec( /** Get file partitions with DPP filtering applied. */ private def getFilePartitions(): Seq[FilePartition] = { + // Use originalPlan.partitionFilters (not partitionFilters) because AQE's + // PlanDynamicPruningFilters may transform InSubqueryExec → Literal.TrueLiteral + // via makeCopy, but originalPlan is not in the active plan tree so it retains + // the original InSubqueryExec needed for DPP filtering. + val dppFilters = originalPlan.partitionFilters + if (bucketedScan) { getDppFilteredBucketedFilePartitions( relation, - partitionFilters, + dppFilters, originalPlan.selectedPartitions, relation.bucketSpec.get, optionalBucketSet, optionalNumCoalescedBuckets) } else { - getDppFilteredFilePartitions(relation, partitionFilters, originalPlan.selectedPartitions) + getDppFilteredFilePartitions(relation, dppFilters, originalPlan.selectedPartitions) } } From 47993d57668b72da0f2f6872785296e1bdc95de0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Feb 2026 15:19:50 -0500 Subject: [PATCH 31/38] Rename NativePlanDataInjector --- native/proto/src/proto/operator.proto | 2 +- .../src/main/scala/org/apache/spark/sql/comet/operators.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 2073f2701e..900900b3c0 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -118,7 +118,7 @@ message NativeScan { // Common data shared across partitions NativeScanCommon common = 1; - // This partition's files only (injected at execution time by NativePlanDataInjector) + // This partition's files only (injected at execution time by NativeScanDataInjector) SparkFilePartition file_partition = 2; } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index dad4ea363e..5d95fd2068 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -82,7 +82,7 @@ private[comet] object PlanDataInjector { // Registry of injectors for different operator types private val injectors: Seq[PlanDataInjector] = - Seq(IcebergPlanDataInjector, NativePlanDataInjector) + Seq(IcebergPlanDataInjector, NativeScanDataInjector) /** * Injects planning data into an Operator tree by finding nodes that need injection and applying @@ -192,7 +192,7 @@ private[comet] object IcebergPlanDataInjector extends PlanDataInjector { /** * Injector for NativeScan operators. */ -private[comet] object NativePlanDataInjector extends PlanDataInjector { +private[comet] object NativeScanDataInjector extends PlanDataInjector { import java.nio.ByteBuffer import java.util.{LinkedHashMap, Map => JMap} From e25798a2ab82da345b9e6d09fb9246ddd18f008f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Feb 2026 16:21:33 -0500 Subject: [PATCH 32/38] minor cleanup --- native/core/src/execution/planner.rs | 2 +- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 4 ++-- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 4 ++-- .../org/apache/spark/sql/comet/shims/ShimCometScanExec.scala | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 0b74555f9b..cb71085025 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -987,7 +987,7 @@ impl PhysicalPlanner { GeneralError("NativeScan missing file_partition data".to_string()) })?; - // Check if this partition has any files (bucketed scan with bucket pruning may have empty partitions) + // Bucketed scan with bucket pruning may produce empty partitions if partition_files.partitioned_file.is_empty() { let empty_exec = Arc::new(EmptyExec::new(required_schema)); return Ok(( diff --git a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 955f3f4eb7..e20365d963 100644 --- a/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, PlanExpression, Predicate} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources._ @@ -159,7 +159,7 @@ trait ShimCometScanExec { } private def isDynamicPruningFilter(e: Expression): Boolean = - e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) + e.exists(_.isInstanceOf[PlanExpression[_]]) protected def newFileScanRDD( fsRelation: HadoopFsRelation, diff --git a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index e066b672c2..f368d1a364 100644 --- a/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -26,7 +26,7 @@ import org.apache.spark.SPARK_VERSION_SHORT import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, PlanExpression, Predicate} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources._ @@ -161,7 +161,7 @@ trait ShimCometScanExec { } private def isDynamicPruningFilter(e: Expression): Boolean = - e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) + e.exists(_.isInstanceOf[PlanExpression[_]]) def isSparkVersionAtLeast355: Boolean = { VersionUtils.majorMinorPatchVersion(SPARK_VERSION_SHORT) match { diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index e98acfff6a..b82c2675a3 100644 --- a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, FileSourceConstantMetadataAttribute, Literal, Predicate} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, FileSourceConstantMetadataAttribute, Literal, PlanExpression, Predicate} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil, ScalarSubquery, ScanFileListing} import org.apache.spark.sql.execution.datasources._ @@ -162,7 +162,7 @@ trait ShimCometScanExec extends ShimStreamSourceAwareSparkPlan { } private def isDynamicPruningFilter(e: Expression): Boolean = - e.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]]) + e.exists(_.isInstanceOf[PlanExpression[_]]) protected def newFileScanRDD( fsRelation: HadoopFsRelation, From b8d46c3e3d8c6d6718ceca4fd205627c34a9df80 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Feb 2026 07:39:25 -0500 Subject: [PATCH 33/38] - Transform SubqueryBroadcast -> BroadcastExchange -> CometColumnarToRow -> CometNativeExec into SubqueryBroadcast -> CometColumnarToRow -> CometBroadcastExchange -> CometNativeExec - This allows CometBroadcastExchange to be reused by both the SubqueryBroadcast path and the join path - CometColumnarToRowExec is still needed because SubqueryBroadcastExec expects HashedRelation from doExecuteBroadcast() --- .../apache/comet/rules/CometExecRule.scala | 70 ++++++++++- .../apache/comet/exec/CometExecSuite.scala | 112 +++++++++++++++++- 2 files changed, 179 insertions(+), 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 8ae32b3c30..c21c0c00e0 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -296,9 +296,77 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } } - plan.transformUp { case op => + val transformedPlan = plan.transformUp { case op => convertNode(op) } + + // Transform SubqueryBroadcastExec inside expressions to enable broadcast exchange reuse. + // SubqueryBroadcast's BroadcastExchange should be transformed to CometBroadcastExchange + // so it can be reused with the join's CometBroadcastExchange. + transformSubqueryBroadcasts(transformedPlan) + } + + /** + * Transforms SubqueryBroadcastExec inside expressions to enable broadcast exchange reuse. + * + * Current structure (doesn't enable reuse): SubqueryBroadcast -> BroadcastExchange -> + * CometColumnarToRow -> CometNativeExec + * + * Transformed structure (enables reuse): SubqueryBroadcast -> CometColumnarToRow -> + * CometBroadcastExchange -> CometNativeExec + * + * The CometBroadcastExchange can then be matched by ReuseExchangeAndSubquery with the join's + * CometBroadcastExchange, enabling broadcast reuse. + */ + private def transformSubqueryBroadcasts(plan: SparkPlan): SparkPlan = { + plan.transformUpWithSubqueries { case p => + p.transformExpressions { case sub: InSubqueryExec => + sub.plan match { + case s: SubqueryBroadcastExec => + transformSubqueryBroadcastExec(s) match { + case Some(newSubquery) => sub.withNewPlan(newSubquery) + case None => sub + } + case _ => sub + } + } + } + } + + /** + * Transforms a SubqueryBroadcastExec to enable broadcast exchange reuse. Returns + * Some(transformed) if transformation was applied, None otherwise. + */ + private def transformSubqueryBroadcastExec( + subquery: SubqueryBroadcastExec): Option[SubqueryBroadcastExec] = { + subquery.child match { + // Pattern: BroadcastExchange -> CometColumnarToRow -> CometNativeExec + case b: BroadcastExchangeExec => + // Unwrap WholeStageCodegenExec if present + val innerChild = b.child match { + case wsc: WholeStageCodegenExec => wsc.child + case other => other + } + innerChild match { + case c2r: CometColumnarToRowExec => + // Unwrap InputAdapter if present (from WholeStageCodegen) + val actualChild = c2r.child match { + case ia: InputAdapter => ia.child + case other => other + } + if (actualChild.isInstanceOf[CometNativeExec]) { + // Transform to: CometColumnarToRow -> CometBroadcastExchange -> CometNativeExec + val nativeChild = actualChild.asInstanceOf[CometNativeExec] + val cometBroadcast = CometBroadcastExchangeExec(b, b.output, b.mode, nativeChild) + val newC2r = CometColumnarToRowExec(cometBroadcast) + Some(subquery.copy(child = newC2r)) + } else { + None + } + case _ => None + } + case _ => None + } } private def normalizePlan(plan: SparkPlan): SparkPlan = { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 8b62b4bb64..1ae831f20f 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -35,10 +35,10 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, He import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateMode, BloomFilterAggregate} import org.apache.spark.sql.comet._ import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} -import org.apache.spark.sql.execution.{CollectLimitExec, ProjectExec, ReusedSubqueryExec, SparkPlan, SQLExecution, SubqueryExec, UnionExec} +import org.apache.spark.sql.execution.{CollectLimitExec, ProjectExec, ReusedSubqueryExec, SparkPlan, SQLExecution, SubqueryBroadcastExec, SubqueryExec, UnionExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, BroadcastExchangeLike, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, CartesianProductExec, SortMergeJoinExec} import org.apache.spark.sql.execution.reuse.ReuseExchangeAndSubquery import org.apache.spark.sql.execution.window.WindowExec @@ -2482,6 +2482,114 @@ class CometExecSuite extends CometTestBase { } } + test("DPP broadcast exchange reuse") { + // Reproduces "partition pruning in broadcast hash joins" from DynamicPartitionPruningSuite. + // When DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY is true, the SubqueryBroadcast + // should reuse the BroadcastExchange from the join via ReusedExchangeExec. + val factData = Seq[(Int, Int, Int, Int)]( + (1030, 3, 2, 10), + (1040, 3, 2, 50), + (1050, 3, 2, 50), + (1060, 3, 2, 50)) + + val storeData = Seq[(Int, String, String)]( + (1, "North-Holland", "NL"), + (2, "South-Holland", "NL"), + (3, "Bavaria", "DE")) + + withTable("fact_stats", "dim_stats") { + factData + .toDF("date_id", "store_id", "product_id", "units_sold") + .write + .partitionBy("store_id") + .format("parquet") + .saveAsTable("fact_stats") + + storeData + .toDF("store_id", "state_province", "country") + .write + .format("parquet") + .saveAsTable("dim_stats") + + // The issue is specific to native_datafusion scan implementation + withSQLConf( + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "true") { + + val df = sql(""" + |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f + |JOIN dim_stats s + |ON f.store_id = s.store_id WHERE s.country = 'DE' + """.stripMargin) + + df.collect() + + val plan = df.queryExecution.executedPlan + // scalastyle:off println + println(s"=== Executed Plan ===\n$plan") + // scalastyle:on println + + // Check that DPP is triggered with SubqueryBroadcast + val subqueryBroadcasts = plan.collectWithSubqueries { case s: SubqueryBroadcastExec => + s + } + assert( + subqueryBroadcasts.nonEmpty, + s"Expected SubqueryBroadcastExec in plan but found none:\n$plan") + + // Check that the SubqueryBroadcast's child is a ReusedExchangeExec + // This is the key assertion - in vanilla Spark this passes, but with Comet it fails + // because SubqueryBroadcast uses BroadcastExchange while the join uses + // CometBroadcastExchange + subqueryBroadcasts.foreach { s => + s.child match { + case _: ReusedExchangeExec => + // Good - the broadcast exchange is being reused + case _: BroadcastQueryStageExec => + // AQE case - also acceptable + case c2r: CometColumnarToRowExec => + // Comet case: SubqueryBroadcast -> CometColumnarToRow -> CometBroadcastExchange + // The CometBroadcastExchange should be reused by the join + c2r.child match { + case cbe: CometBroadcastExchangeExec => + val hasReuse = plan.exists { + case ReusedExchangeExec(_, e: CometBroadcastExchangeExec) => + e.canonicalized == cbe.canonicalized + case _ => false + } + assert( + hasReuse, + s"CometBroadcastExchange should be reused.\n" + + s"SubqueryBroadcast structure: ${s.child.getClass.getSimpleName} -> " + + s"${cbe.getClass.getSimpleName}\nFull plan:\n$plan") + case other => + fail( + s"Expected CometBroadcastExchangeExec under CometColumnarToRowExec, " + + s"got ${other.getClass.getSimpleName}\n$plan") + } + case b: BroadcastExchangeLike => + val hasReuse = plan.exists { + case ReusedExchangeExec(_, e) => e eq b + case _ => false + } + assert( + hasReuse, + s"SubqueryBroadcast's BroadcastExchange should have been reused.\n" + + s"SubqueryBroadcast child: ${s.child.getClass.getSimpleName}\n" + + s"Full plan:\n$plan") + case other => + fail(s"Unexpected SubqueryBroadcast child: ${other.getClass.getSimpleName}\n$plan") + } + } + + checkSparkAnswer(df) + } + } + } + } case class BucketedTableTestSpec( From d14c440992e529890d4a875302c25fcfc44e119a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Feb 2026 09:29:34 -0500 Subject: [PATCH 34/38] - Changed transformUpWithSubqueries -> transformUp to preserve ReusedSubqueryExec. - New plans. --- .../apache/comet/rules/CometExecRule.scala | 5 +- .../q1.native_datafusion/extended.txt | 6 +- .../q1.native_iceberg_compat/extended.txt | 6 +- .../q10.native_datafusion/extended.txt | 6 +- .../q10.native_iceberg_compat/extended.txt | 6 +- .../q11.native_datafusion/extended.txt | 10 +- .../q11.native_iceberg_compat/extended.txt | 10 +- .../q12.native_datafusion/extended.txt | 6 +- .../q12.native_iceberg_compat/extended.txt | 6 +- .../q13.native_datafusion/extended.txt | 6 +- .../q13.native_iceberg_compat/extended.txt | 6 +- .../q14a.native_datafusion/extended.txt | 30 ++-- .../q14a.native_iceberg_compat/extended.txt | 30 ++-- .../q14b.native_datafusion/extended.txt | 38 ++-- .../q14b.native_iceberg_compat/extended.txt | 38 ++-- .../q15.native_datafusion/extended.txt | 6 +- .../q15.native_iceberg_compat/extended.txt | 6 +- .../q17.native_datafusion/extended.txt | 10 +- .../q17.native_iceberg_compat/extended.txt | 10 +- .../q18.native_datafusion/extended.txt | 6 +- .../q18.native_iceberg_compat/extended.txt | 6 +- .../q20.native_datafusion/extended.txt | 6 +- .../q20.native_iceberg_compat/extended.txt | 6 +- .../q21.native_datafusion/extended.txt | 6 +- .../q21.native_iceberg_compat/extended.txt | 6 +- .../q22.native_datafusion/extended.txt | 6 +- .../q22.native_iceberg_compat/extended.txt | 6 +- .../q23a.native_datafusion/extended.txt | 18 +- .../q23a.native_iceberg_compat/extended.txt | 18 +- .../q23b.native_datafusion/extended.txt | 18 +- .../q23b.native_iceberg_compat/extended.txt | 18 +- .../q25.native_datafusion/extended.txt | 10 +- .../q25.native_iceberg_compat/extended.txt | 10 +- .../q26.native_datafusion/extended.txt | 6 +- .../q26.native_iceberg_compat/extended.txt | 6 +- .../q27.native_datafusion/extended.txt | 6 +- .../q27.native_iceberg_compat/extended.txt | 6 +- .../q29.native_datafusion/extended.txt | 14 +- .../q29.native_iceberg_compat/extended.txt | 14 +- .../q30.native_datafusion/extended.txt | 6 +- .../q30.native_iceberg_compat/extended.txt | 6 +- .../q31.native_datafusion/extended.txt | 14 +- .../q31.native_iceberg_compat/extended.txt | 14 +- .../q32.native_datafusion/extended.txt | 6 +- .../q32.native_iceberg_compat/extended.txt | 6 +- .../q33.native_datafusion/extended.txt | 6 +- .../q33.native_iceberg_compat/extended.txt | 6 +- .../q34.native_datafusion/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 6 +- .../q35.native_datafusion/extended.txt | 6 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q36.native_datafusion/extended.txt | 6 +- .../q36.native_iceberg_compat/extended.txt | 6 +- .../q37.native_datafusion/extended.txt | 6 +- .../q37.native_iceberg_compat/extended.txt | 6 +- .../q38.native_datafusion/extended.txt | 6 +- .../q38.native_iceberg_compat/extended.txt | 6 +- .../q39a.native_datafusion/extended.txt | 10 +- .../q39a.native_iceberg_compat/extended.txt | 10 +- .../q39b.native_datafusion/extended.txt | 10 +- .../q39b.native_iceberg_compat/extended.txt | 10 +- .../q4.native_datafusion/extended.txt | 10 +- .../q4.native_iceberg_compat/extended.txt | 10 +- .../q40.native_datafusion/extended.txt | 6 +- .../q40.native_iceberg_compat/extended.txt | 6 +- .../q45.native_datafusion/extended.txt | 6 +- .../q45.native_iceberg_compat/extended.txt | 6 +- .../q46.native_datafusion/extended.txt | 6 +- .../q46.native_iceberg_compat/extended.txt | 6 +- .../q47.native_datafusion/extended.txt | 14 +- .../q47.native_iceberg_compat/extended.txt | 14 +- .../q48.native_datafusion/extended.txt | 6 +- .../q48.native_iceberg_compat/extended.txt | 6 +- .../q49.native_datafusion/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q5.native_datafusion/extended.txt | 6 +- .../q5.native_iceberg_compat/extended.txt | 6 +- .../q50.native_datafusion/extended.txt | 6 +- .../q50.native_iceberg_compat/extended.txt | 6 +- .../q51.native_datafusion/extended.txt | 6 +- .../q51.native_iceberg_compat/extended.txt | 6 +- .../q53.native_datafusion/extended.txt | 6 +- .../q53.native_iceberg_compat/extended.txt | 6 +- .../q54.native_datafusion/extended.txt | 28 ++- .../q54.native_iceberg_compat/extended.txt | 28 +-- .../q56.native_datafusion/extended.txt | 6 +- .../q56.native_iceberg_compat/extended.txt | 6 +- .../q57.native_datafusion/extended.txt | 14 +- .../q57.native_iceberg_compat/extended.txt | 14 +- .../q58.native_datafusion/extended.txt | 6 +- .../q58.native_iceberg_compat/extended.txt | 6 +- .../q6.native_datafusion/extended.txt | 15 +- .../q6.native_iceberg_compat/extended.txt | 15 +- .../q60.native_datafusion/extended.txt | 6 +- .../q60.native_iceberg_compat/extended.txt | 6 +- .../q61.native_datafusion/extended.txt | 6 +- .../q61.native_iceberg_compat/extended.txt | 6 +- .../q63.native_datafusion/extended.txt | 6 +- .../q63.native_iceberg_compat/extended.txt | 6 +- .../q64.native_datafusion/extended.txt | 10 +- .../q64.native_iceberg_compat/extended.txt | 10 +- .../q65.native_datafusion/extended.txt | 6 +- .../q65.native_iceberg_compat/extended.txt | 6 +- .../q66.native_datafusion/extended.txt | 6 +- .../q66.native_iceberg_compat/extended.txt | 6 +- .../q67.native_datafusion/extended.txt | 6 +- .../q67.native_iceberg_compat/extended.txt | 6 +- .../q68.native_datafusion/extended.txt | 6 +- .../q68.native_iceberg_compat/extended.txt | 6 +- .../q69.native_datafusion/extended.txt | 6 +- .../q69.native_iceberg_compat/extended.txt | 6 +- .../q7.native_datafusion/extended.txt | 6 +- .../q7.native_iceberg_compat/extended.txt | 6 +- .../q70.native_datafusion/extended.txt | 6 +- .../q70.native_iceberg_compat/extended.txt | 6 +- .../q71.native_datafusion/extended.txt | 6 +- .../q71.native_iceberg_compat/extended.txt | 6 +- .../q72.native_datafusion/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 6 +- .../q73.native_datafusion/extended.txt | 6 +- .../q73.native_iceberg_compat/extended.txt | 6 +- .../q74.native_datafusion/extended.txt | 10 +- .../q74.native_iceberg_compat/extended.txt | 10 +- .../q75.native_datafusion/extended.txt | 10 +- .../q75.native_iceberg_compat/extended.txt | 10 +- .../q77.native_datafusion/extended.txt | 6 +- .../q77.native_iceberg_compat/extended.txt | 6 +- .../q78.native_datafusion/extended.txt | 6 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q79.native_datafusion/extended.txt | 6 +- .../q79.native_iceberg_compat/extended.txt | 6 +- .../q8.native_datafusion/extended.txt | 6 +- .../q8.native_iceberg_compat/extended.txt | 6 +- .../q80.native_datafusion/extended.txt | 6 +- .../q80.native_iceberg_compat/extended.txt | 6 +- .../q81.native_datafusion/extended.txt | 6 +- .../q81.native_iceberg_compat/extended.txt | 6 +- .../q82.native_datafusion/extended.txt | 6 +- .../q82.native_iceberg_compat/extended.txt | 6 +- .../q83.native_datafusion/extended.txt | 6 +- .../q83.native_iceberg_compat/extended.txt | 6 +- .../q85.native_datafusion/extended.txt | 6 +- .../q85.native_iceberg_compat/extended.txt | 6 +- .../q86.native_datafusion/extended.txt | 6 +- .../q86.native_iceberg_compat/extended.txt | 6 +- .../q87.native_datafusion/extended.txt | 6 +- .../q87.native_iceberg_compat/extended.txt | 6 +- .../q89.native_datafusion/extended.txt | 6 +- .../q89.native_iceberg_compat/extended.txt | 6 +- .../q91.native_datafusion/extended.txt | 6 +- .../q91.native_iceberg_compat/extended.txt | 6 +- .../q92.native_datafusion/extended.txt | 6 +- .../q92.native_iceberg_compat/extended.txt | 6 +- .../q97.native_datafusion/extended.txt | 6 +- .../q97.native_iceberg_compat/extended.txt | 6 +- .../q98.native_datafusion/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 6 +- .../q1.native_datafusion/extended.txt | 6 +- .../q1.native_iceberg_compat/extended.txt | 6 +- .../q10.native_datafusion/extended.txt | 6 +- .../q10.native_iceberg_compat/extended.txt | 6 +- .../q11.native_datafusion/extended.txt | 10 +- .../q11.native_iceberg_compat/extended.txt | 10 +- .../q12.native_datafusion/extended.txt | 6 +- .../q12.native_iceberg_compat/extended.txt | 6 +- .../q13.native_datafusion/extended.txt | 6 +- .../q13.native_iceberg_compat/extended.txt | 6 +- .../q14a.native_datafusion/extended.txt | 30 ++-- .../q14a.native_iceberg_compat/extended.txt | 30 ++-- .../q14b.native_datafusion/extended.txt | 38 ++-- .../q14b.native_iceberg_compat/extended.txt | 38 ++-- .../q15.native_datafusion/extended.txt | 6 +- .../q15.native_iceberg_compat/extended.txt | 6 +- .../q17.native_datafusion/extended.txt | 10 +- .../q17.native_iceberg_compat/extended.txt | 10 +- .../q18.native_datafusion/extended.txt | 6 +- .../q18.native_iceberg_compat/extended.txt | 6 +- .../q20.native_datafusion/extended.txt | 6 +- .../q20.native_iceberg_compat/extended.txt | 6 +- .../q21.native_datafusion/extended.txt | 6 +- .../q21.native_iceberg_compat/extended.txt | 6 +- .../q22.native_datafusion/extended.txt | 6 +- .../q22.native_iceberg_compat/extended.txt | 6 +- .../q23a.native_datafusion/extended.txt | 18 +- .../q23a.native_iceberg_compat/extended.txt | 18 +- .../q23b.native_datafusion/extended.txt | 18 +- .../q23b.native_iceberg_compat/extended.txt | 18 +- .../q25.native_datafusion/extended.txt | 10 +- .../q25.native_iceberg_compat/extended.txt | 10 +- .../q26.native_datafusion/extended.txt | 6 +- .../q26.native_iceberg_compat/extended.txt | 6 +- .../q27.native_datafusion/extended.txt | 6 +- .../q27.native_iceberg_compat/extended.txt | 6 +- .../q29.native_datafusion/extended.txt | 14 +- .../q29.native_iceberg_compat/extended.txt | 14 +- .../q30.native_datafusion/extended.txt | 6 +- .../q30.native_iceberg_compat/extended.txt | 6 +- .../q31.native_datafusion/extended.txt | 14 +- .../q31.native_iceberg_compat/extended.txt | 14 +- .../q32.native_datafusion/extended.txt | 6 +- .../q32.native_iceberg_compat/extended.txt | 6 +- .../q33.native_datafusion/extended.txt | 6 +- .../q33.native_iceberg_compat/extended.txt | 6 +- .../q34.native_datafusion/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 6 +- .../q35.native_datafusion/extended.txt | 6 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q36.native_datafusion/extended.txt | 6 +- .../q36.native_iceberg_compat/extended.txt | 6 +- .../q37.native_datafusion/extended.txt | 6 +- .../q37.native_iceberg_compat/extended.txt | 6 +- .../q38.native_datafusion/extended.txt | 6 +- .../q38.native_iceberg_compat/extended.txt | 6 +- .../q39a.native_datafusion/extended.txt | 10 +- .../q39a.native_iceberg_compat/extended.txt | 10 +- .../q39b.native_datafusion/extended.txt | 10 +- .../q39b.native_iceberg_compat/extended.txt | 10 +- .../q4.native_datafusion/extended.txt | 10 +- .../q4.native_iceberg_compat/extended.txt | 10 +- .../q40.native_datafusion/extended.txt | 6 +- .../q40.native_iceberg_compat/extended.txt | 6 +- .../q45.native_datafusion/extended.txt | 6 +- .../q45.native_iceberg_compat/extended.txt | 6 +- .../q46.native_datafusion/extended.txt | 6 +- .../q46.native_iceberg_compat/extended.txt | 6 +- .../q47.native_datafusion/extended.txt | 14 +- .../q47.native_iceberg_compat/extended.txt | 14 +- .../q48.native_datafusion/extended.txt | 6 +- .../q48.native_iceberg_compat/extended.txt | 6 +- .../q49.native_datafusion/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q5.native_datafusion/extended.txt | 6 +- .../q5.native_iceberg_compat/extended.txt | 6 +- .../q50.native_datafusion/extended.txt | 6 +- .../q50.native_iceberg_compat/extended.txt | 6 +- .../q51.native_datafusion/extended.txt | 6 +- .../q51.native_iceberg_compat/extended.txt | 6 +- .../q53.native_datafusion/extended.txt | 6 +- .../q53.native_iceberg_compat/extended.txt | 6 +- .../q54.native_datafusion/extended.txt | 28 ++- .../q54.native_iceberg_compat/extended.txt | 28 +-- .../q56.native_datafusion/extended.txt | 6 +- .../q56.native_iceberg_compat/extended.txt | 6 +- .../q57.native_datafusion/extended.txt | 14 +- .../q57.native_iceberg_compat/extended.txt | 14 +- .../q58.native_datafusion/extended.txt | 6 +- .../q58.native_iceberg_compat/extended.txt | 6 +- .../q6.native_datafusion/extended.txt | 15 +- .../q6.native_iceberg_compat/extended.txt | 15 +- .../q60.native_datafusion/extended.txt | 6 +- .../q60.native_iceberg_compat/extended.txt | 6 +- .../q61.native_datafusion/extended.txt | 6 +- .../q61.native_iceberg_compat/extended.txt | 6 +- .../q63.native_datafusion/extended.txt | 6 +- .../q63.native_iceberg_compat/extended.txt | 6 +- .../q64.native_datafusion/extended.txt | 10 +- .../q64.native_iceberg_compat/extended.txt | 10 +- .../q65.native_datafusion/extended.txt | 6 +- .../q65.native_iceberg_compat/extended.txt | 6 +- .../q66.native_datafusion/extended.txt | 6 +- .../q66.native_iceberg_compat/extended.txt | 6 +- .../q67.native_datafusion/extended.txt | 6 +- .../q67.native_iceberg_compat/extended.txt | 6 +- .../q68.native_datafusion/extended.txt | 6 +- .../q68.native_iceberg_compat/extended.txt | 6 +- .../q69.native_datafusion/extended.txt | 6 +- .../q69.native_iceberg_compat/extended.txt | 6 +- .../q7.native_datafusion/extended.txt | 6 +- .../q7.native_iceberg_compat/extended.txt | 6 +- .../q70.native_datafusion/extended.txt | 6 +- .../q70.native_iceberg_compat/extended.txt | 6 +- .../q71.native_datafusion/extended.txt | 6 +- .../q71.native_iceberg_compat/extended.txt | 6 +- .../q72.native_datafusion/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 6 +- .../q73.native_datafusion/extended.txt | 6 +- .../q73.native_iceberg_compat/extended.txt | 6 +- .../q74.native_datafusion/extended.txt | 10 +- .../q74.native_iceberg_compat/extended.txt | 10 +- .../q75.native_datafusion/extended.txt | 10 +- .../q75.native_iceberg_compat/extended.txt | 10 +- .../q77.native_datafusion/extended.txt | 6 +- .../q77.native_iceberg_compat/extended.txt | 6 +- .../q78.native_datafusion/extended.txt | 6 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q79.native_datafusion/extended.txt | 6 +- .../q79.native_iceberg_compat/extended.txt | 6 +- .../q8.native_datafusion/extended.txt | 6 +- .../q8.native_iceberg_compat/extended.txt | 6 +- .../q80.native_datafusion/extended.txt | 6 +- .../q80.native_iceberg_compat/extended.txt | 6 +- .../q81.native_datafusion/extended.txt | 6 +- .../q81.native_iceberg_compat/extended.txt | 6 +- .../q82.native_datafusion/extended.txt | 6 +- .../q82.native_iceberg_compat/extended.txt | 6 +- .../q83.ansi.native_datafusion/extended.txt | 6 +- .../extended.txt | 6 +- .../q85.native_datafusion/extended.txt | 6 +- .../q85.native_iceberg_compat/extended.txt | 6 +- .../q86.native_datafusion/extended.txt | 6 +- .../q86.native_iceberg_compat/extended.txt | 6 +- .../q87.native_datafusion/extended.txt | 6 +- .../q87.native_iceberg_compat/extended.txt | 6 +- .../q89.native_datafusion/extended.txt | 6 +- .../q89.native_iceberg_compat/extended.txt | 6 +- .../q91.native_datafusion/extended.txt | 6 +- .../q91.native_iceberg_compat/extended.txt | 6 +- .../q92.native_datafusion/extended.txt | 6 +- .../q92.native_iceberg_compat/extended.txt | 6 +- .../q97.native_datafusion/extended.txt | 6 +- .../q97.native_iceberg_compat/extended.txt | 6 +- .../q98.native_datafusion/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 6 +- .../q1.native_datafusion/extended.txt | 6 +- .../q1.native_iceberg_compat/extended.txt | 6 +- .../q10.native_datafusion/extended.txt | 6 +- .../q10.native_iceberg_compat/extended.txt | 6 +- .../q11.native_datafusion/extended.txt | 10 +- .../q11.native_iceberg_compat/extended.txt | 10 +- .../q12.native_datafusion/extended.txt | 6 +- .../q12.native_iceberg_compat/extended.txt | 6 +- .../q13.native_datafusion/extended.txt | 6 +- .../q13.native_iceberg_compat/extended.txt | 6 +- .../q14a.native_datafusion/extended.txt | 30 ++-- .../q14a.native_iceberg_compat/extended.txt | 30 ++-- .../q14b.native_datafusion/extended.txt | 38 ++-- .../q14b.native_iceberg_compat/extended.txt | 38 ++-- .../q15.native_datafusion/extended.txt | 6 +- .../q15.native_iceberg_compat/extended.txt | 6 +- .../q17.native_datafusion/extended.txt | 10 +- .../q17.native_iceberg_compat/extended.txt | 10 +- .../q18.native_datafusion/extended.txt | 6 +- .../q18.native_iceberg_compat/extended.txt | 6 +- .../q20.native_datafusion/extended.txt | 6 +- .../q20.native_iceberg_compat/extended.txt | 6 +- .../q21.native_datafusion/extended.txt | 6 +- .../q21.native_iceberg_compat/extended.txt | 6 +- .../q22.native_datafusion/extended.txt | 6 +- .../q22.native_iceberg_compat/extended.txt | 6 +- .../q23a.native_datafusion/extended.txt | 18 +- .../q23a.native_iceberg_compat/extended.txt | 18 +- .../q23b.native_datafusion/extended.txt | 18 +- .../q23b.native_iceberg_compat/extended.txt | 18 +- .../q25.native_datafusion/extended.txt | 10 +- .../q25.native_iceberg_compat/extended.txt | 10 +- .../q26.native_datafusion/extended.txt | 6 +- .../q26.native_iceberg_compat/extended.txt | 6 +- .../q27.native_datafusion/extended.txt | 6 +- .../q27.native_iceberg_compat/extended.txt | 6 +- .../q29.native_datafusion/extended.txt | 14 +- .../q29.native_iceberg_compat/extended.txt | 14 +- .../q30.native_datafusion/extended.txt | 6 +- .../q30.native_iceberg_compat/extended.txt | 6 +- .../q31.native_datafusion/extended.txt | 14 +- .../q31.native_iceberg_compat/extended.txt | 14 +- .../q32.native_datafusion/extended.txt | 6 +- .../q32.native_iceberg_compat/extended.txt | 6 +- .../q33.native_datafusion/extended.txt | 6 +- .../q33.native_iceberg_compat/extended.txt | 6 +- .../q34.native_datafusion/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 6 +- .../q35.native_datafusion/extended.txt | 6 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q36.native_datafusion/extended.txt | 6 +- .../q36.native_iceberg_compat/extended.txt | 6 +- .../q37.native_datafusion/extended.txt | 6 +- .../q37.native_iceberg_compat/extended.txt | 6 +- .../q38.native_datafusion/extended.txt | 6 +- .../q38.native_iceberg_compat/extended.txt | 6 +- .../q39a.native_datafusion/extended.txt | 10 +- .../q39a.native_iceberg_compat/extended.txt | 10 +- .../q39b.native_datafusion/extended.txt | 10 +- .../q39b.native_iceberg_compat/extended.txt | 10 +- .../q4.native_datafusion/extended.txt | 10 +- .../q4.native_iceberg_compat/extended.txt | 10 +- .../q40.native_datafusion/extended.txt | 6 +- .../q40.native_iceberg_compat/extended.txt | 6 +- .../q45.native_datafusion/extended.txt | 6 +- .../q45.native_iceberg_compat/extended.txt | 6 +- .../q46.native_datafusion/extended.txt | 6 +- .../q46.native_iceberg_compat/extended.txt | 6 +- .../q47.native_datafusion/extended.txt | 14 +- .../q47.native_iceberg_compat/extended.txt | 14 +- .../q48.native_datafusion/extended.txt | 6 +- .../q48.native_iceberg_compat/extended.txt | 6 +- .../q49.native_datafusion/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q5.native_datafusion/extended.txt | 6 +- .../q5.native_iceberg_compat/extended.txt | 6 +- .../q50.native_datafusion/extended.txt | 6 +- .../q50.native_iceberg_compat/extended.txt | 6 +- .../q51.native_datafusion/extended.txt | 6 +- .../q51.native_iceberg_compat/extended.txt | 6 +- .../q53.native_datafusion/extended.txt | 6 +- .../q53.native_iceberg_compat/extended.txt | 6 +- .../q54.native_datafusion/extended.txt | 28 ++- .../q54.native_iceberg_compat/extended.txt | 28 +-- .../q56.native_datafusion/extended.txt | 6 +- .../q56.native_iceberg_compat/extended.txt | 6 +- .../q57.native_datafusion/extended.txt | 14 +- .../q57.native_iceberg_compat/extended.txt | 14 +- .../q58.native_datafusion/extended.txt | 10 +- .../q58.native_iceberg_compat/extended.txt | 10 +- .../q6.native_datafusion/extended.txt | 15 +- .../q6.native_iceberg_compat/extended.txt | 15 +- .../q60.native_datafusion/extended.txt | 6 +- .../q60.native_iceberg_compat/extended.txt | 6 +- .../q61.native_datafusion/extended.txt | 6 +- .../q61.native_iceberg_compat/extended.txt | 6 +- .../q63.native_datafusion/extended.txt | 6 +- .../q63.native_iceberg_compat/extended.txt | 6 +- .../q64.native_datafusion/extended.txt | 10 +- .../q64.native_iceberg_compat/extended.txt | 10 +- .../q65.native_datafusion/extended.txt | 6 +- .../q65.native_iceberg_compat/extended.txt | 6 +- .../q66.native_datafusion/extended.txt | 6 +- .../q66.native_iceberg_compat/extended.txt | 6 +- .../q67.native_datafusion/extended.txt | 6 +- .../q67.native_iceberg_compat/extended.txt | 6 +- .../q68.native_datafusion/extended.txt | 6 +- .../q68.native_iceberg_compat/extended.txt | 6 +- .../q69.native_datafusion/extended.txt | 6 +- .../q69.native_iceberg_compat/extended.txt | 6 +- .../q7.native_datafusion/extended.txt | 6 +- .../q7.native_iceberg_compat/extended.txt | 6 +- .../q70.native_datafusion/extended.txt | 6 +- .../q70.native_iceberg_compat/extended.txt | 6 +- .../q71.native_datafusion/extended.txt | 6 +- .../q71.native_iceberg_compat/extended.txt | 6 +- .../q72.native_datafusion/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 6 +- .../q73.native_datafusion/extended.txt | 6 +- .../q73.native_iceberg_compat/extended.txt | 6 +- .../q74.native_datafusion/extended.txt | 10 +- .../q74.native_iceberg_compat/extended.txt | 10 +- .../q75.native_datafusion/extended.txt | 10 +- .../q75.native_iceberg_compat/extended.txt | 10 +- .../q77.native_datafusion/extended.txt | 6 +- .../q77.native_iceberg_compat/extended.txt | 6 +- .../q78.native_datafusion/extended.txt | 6 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q79.native_datafusion/extended.txt | 6 +- .../q79.native_iceberg_compat/extended.txt | 6 +- .../q8.native_datafusion/extended.txt | 6 +- .../q8.native_iceberg_compat/extended.txt | 6 +- .../q80.native_datafusion/extended.txt | 6 +- .../q80.native_iceberg_compat/extended.txt | 6 +- .../q81.native_datafusion/extended.txt | 6 +- .../q81.native_iceberg_compat/extended.txt | 6 +- .../q82.native_datafusion/extended.txt | 6 +- .../q82.native_iceberg_compat/extended.txt | 6 +- .../q83.native_datafusion/extended.txt | 10 +- .../q83.native_iceberg_compat/extended.txt | 10 +- .../q85.native_datafusion/extended.txt | 6 +- .../q85.native_iceberg_compat/extended.txt | 6 +- .../q86.native_datafusion/extended.txt | 6 +- .../q86.native_iceberg_compat/extended.txt | 6 +- .../q87.native_datafusion/extended.txt | 6 +- .../q87.native_iceberg_compat/extended.txt | 6 +- .../q89.native_datafusion/extended.txt | 6 +- .../q89.native_iceberg_compat/extended.txt | 6 +- .../q91.native_datafusion/extended.txt | 6 +- .../q91.native_iceberg_compat/extended.txt | 6 +- .../q92.native_datafusion/extended.txt | 6 +- .../q92.native_iceberg_compat/extended.txt | 6 +- .../q97.native_datafusion/extended.txt | 6 +- .../q97.native_iceberg_compat/extended.txt | 6 +- .../q98.native_datafusion/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 6 +- .../q10a.native_datafusion/extended.txt | 6 +- .../q10a.native_iceberg_compat/extended.txt | 6 +- .../q11.native_datafusion/extended.txt | 10 +- .../q11.native_iceberg_compat/extended.txt | 10 +- .../q12.native_datafusion/extended.txt | 6 +- .../q12.native_iceberg_compat/extended.txt | 6 +- .../q14.native_datafusion/extended.txt | 38 ++-- .../q14.native_iceberg_compat/extended.txt | 38 ++-- .../q14a.native_datafusion/extended.txt | 162 +++++++++--------- .../q14a.native_iceberg_compat/extended.txt | 162 +++++++++--------- .../q18a.native_datafusion/extended.txt | 6 +- .../q18a.native_iceberg_compat/extended.txt | 6 +- .../q20.native_datafusion/extended.txt | 6 +- .../q20.native_iceberg_compat/extended.txt | 6 +- .../q22.native_datafusion/extended.txt | 6 +- .../q22.native_iceberg_compat/extended.txt | 6 +- .../q22a.native_datafusion/extended.txt | 22 +-- .../q22a.native_iceberg_compat/extended.txt | 22 +-- .../q27a.native_datafusion/extended.txt | 6 +- .../q27a.native_iceberg_compat/extended.txt | 6 +- .../q34.native_datafusion/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 6 +- .../q35.native_datafusion/extended.txt | 6 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q35a.native_datafusion/extended.txt | 6 +- .../q35a.native_iceberg_compat/extended.txt | 6 +- .../q36a.native_datafusion/extended.txt | 14 +- .../q36a.native_iceberg_compat/extended.txt | 14 +- .../q47.native_datafusion/extended.txt | 14 +- .../q47.native_iceberg_compat/extended.txt | 14 +- .../q49.native_datafusion/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q51a.native_datafusion/extended.txt | 18 +- .../q51a.native_iceberg_compat/extended.txt | 18 +- .../q57.native_datafusion/extended.txt | 14 +- .../q57.native_iceberg_compat/extended.txt | 14 +- .../q5a.native_datafusion/extended.txt | 14 +- .../q5a.native_iceberg_compat/extended.txt | 14 +- .../q6.native_datafusion/extended.txt | 15 +- .../q6.native_iceberg_compat/extended.txt | 15 +- .../q64.native_datafusion/extended.txt | 10 +- .../q64.native_iceberg_compat/extended.txt | 10 +- .../q67a.native_datafusion/extended.txt | 38 ++-- .../q67a.native_iceberg_compat/extended.txt | 38 ++-- .../q70a.native_datafusion/extended.txt | 14 +- .../q70a.native_iceberg_compat/extended.txt | 14 +- .../q72.native_datafusion/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 6 +- .../q74.native_datafusion/extended.txt | 10 +- .../q74.native_iceberg_compat/extended.txt | 10 +- .../q75.native_datafusion/extended.txt | 10 +- .../q75.native_iceberg_compat/extended.txt | 10 +- .../q77a.native_datafusion/extended.txt | 14 +- .../q77a.native_iceberg_compat/extended.txt | 14 +- .../q78.native_datafusion/extended.txt | 6 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q80a.native_datafusion/extended.txt | 14 +- .../q80a.native_iceberg_compat/extended.txt | 14 +- .../q86a.native_datafusion/extended.txt | 14 +- .../q86a.native_iceberg_compat/extended.txt | 14 +- .../q98.native_datafusion/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 6 +- .../q10a.native_datafusion/extended.txt | 6 +- .../q10a.native_iceberg_compat/extended.txt | 6 +- .../q11.native_datafusion/extended.txt | 10 +- .../q11.native_iceberg_compat/extended.txt | 10 +- .../q12.native_datafusion/extended.txt | 6 +- .../q12.native_iceberg_compat/extended.txt | 6 +- .../q14.native_datafusion/extended.txt | 38 ++-- .../q14.native_iceberg_compat/extended.txt | 38 ++-- .../q14a.native_datafusion/extended.txt | 162 +++++++++--------- .../q14a.native_iceberg_compat/extended.txt | 162 +++++++++--------- .../q18a.native_datafusion/extended.txt | 6 +- .../q18a.native_iceberg_compat/extended.txt | 6 +- .../q20.native_datafusion/extended.txt | 6 +- .../q20.native_iceberg_compat/extended.txt | 6 +- .../q22.native_datafusion/extended.txt | 6 +- .../q22.native_iceberg_compat/extended.txt | 6 +- .../q22a.native_datafusion/extended.txt | 22 +-- .../q22a.native_iceberg_compat/extended.txt | 22 +-- .../q27a.native_datafusion/extended.txt | 6 +- .../q27a.native_iceberg_compat/extended.txt | 6 +- .../q34.native_datafusion/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 6 +- .../q35.native_datafusion/extended.txt | 6 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q35a.native_datafusion/extended.txt | 6 +- .../q35a.native_iceberg_compat/extended.txt | 6 +- .../q36a.native_datafusion/extended.txt | 14 +- .../q36a.native_iceberg_compat/extended.txt | 14 +- .../q47.native_datafusion/extended.txt | 14 +- .../q47.native_iceberg_compat/extended.txt | 14 +- .../q49.native_datafusion/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q51a.native_datafusion/extended.txt | 18 +- .../q51a.native_iceberg_compat/extended.txt | 18 +- .../q57.native_datafusion/extended.txt | 14 +- .../q57.native_iceberg_compat/extended.txt | 14 +- .../q5a.native_datafusion/extended.txt | 14 +- .../q5a.native_iceberg_compat/extended.txt | 14 +- .../q6.native_datafusion/extended.txt | 15 +- .../q6.native_iceberg_compat/extended.txt | 15 +- .../q64.native_datafusion/extended.txt | 10 +- .../q64.native_iceberg_compat/extended.txt | 10 +- .../q67a.native_datafusion/extended.txt | 38 ++-- .../q67a.native_iceberg_compat/extended.txt | 38 ++-- .../q70a.native_datafusion/extended.txt | 14 +- .../q70a.native_iceberg_compat/extended.txt | 14 +- .../q72.native_datafusion/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 6 +- .../q74.native_datafusion/extended.txt | 10 +- .../q74.native_iceberg_compat/extended.txt | 10 +- .../q75.native_datafusion/extended.txt | 10 +- .../q75.native_iceberg_compat/extended.txt | 10 +- .../q77a.native_datafusion/extended.txt | 14 +- .../q77a.native_iceberg_compat/extended.txt | 14 +- .../q78.native_datafusion/extended.txt | 6 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q80a.native_datafusion/extended.txt | 14 +- .../q80a.native_iceberg_compat/extended.txt | 14 +- .../q86a.native_datafusion/extended.txt | 14 +- .../q86a.native_iceberg_compat/extended.txt | 14 +- .../q98.native_datafusion/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 6 +- .../q10a.native_datafusion/extended.txt | 6 +- .../q10a.native_iceberg_compat/extended.txt | 6 +- .../q11.native_datafusion/extended.txt | 10 +- .../q11.native_iceberg_compat/extended.txt | 10 +- .../q12.native_datafusion/extended.txt | 6 +- .../q12.native_iceberg_compat/extended.txt | 6 +- .../q14.native_datafusion/extended.txt | 38 ++-- .../q14.native_iceberg_compat/extended.txt | 38 ++-- .../q14a.native_datafusion/extended.txt | 162 +++++++++--------- .../q14a.native_iceberg_compat/extended.txt | 162 +++++++++--------- .../q18a.native_datafusion/extended.txt | 6 +- .../q18a.native_iceberg_compat/extended.txt | 6 +- .../q20.native_datafusion/extended.txt | 6 +- .../q20.native_iceberg_compat/extended.txt | 6 +- .../q22.native_datafusion/extended.txt | 6 +- .../q22.native_iceberg_compat/extended.txt | 6 +- .../q22a.native_datafusion/extended.txt | 22 +-- .../q22a.native_iceberg_compat/extended.txt | 22 +-- .../q27a.native_datafusion/extended.txt | 6 +- .../q27a.native_iceberg_compat/extended.txt | 6 +- .../q34.native_datafusion/extended.txt | 6 +- .../q34.native_iceberg_compat/extended.txt | 6 +- .../q35.native_datafusion/extended.txt | 6 +- .../q35.native_iceberg_compat/extended.txt | 6 +- .../q35a.native_datafusion/extended.txt | 6 +- .../q35a.native_iceberg_compat/extended.txt | 6 +- .../q36a.native_datafusion/extended.txt | 14 +- .../q36a.native_iceberg_compat/extended.txt | 14 +- .../q47.native_datafusion/extended.txt | 14 +- .../q47.native_iceberg_compat/extended.txt | 14 +- .../q49.native_datafusion/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q51a.native_datafusion/extended.txt | 18 +- .../q51a.native_iceberg_compat/extended.txt | 18 +- .../q57.native_datafusion/extended.txt | 14 +- .../q57.native_iceberg_compat/extended.txt | 14 +- .../q5a.native_datafusion/extended.txt | 14 +- .../q5a.native_iceberg_compat/extended.txt | 14 +- .../q6.native_datafusion/extended.txt | 15 +- .../q6.native_iceberg_compat/extended.txt | 15 +- .../q64.native_datafusion/extended.txt | 10 +- .../q64.native_iceberg_compat/extended.txt | 10 +- .../q67a.native_datafusion/extended.txt | 38 ++-- .../q67a.native_iceberg_compat/extended.txt | 38 ++-- .../q70a.native_datafusion/extended.txt | 14 +- .../q70a.native_iceberg_compat/extended.txt | 14 +- .../q72.native_datafusion/extended.txt | 6 +- .../q72.native_iceberg_compat/extended.txt | 6 +- .../q74.native_datafusion/extended.txt | 10 +- .../q74.native_iceberg_compat/extended.txt | 10 +- .../q75.native_datafusion/extended.txt | 10 +- .../q75.native_iceberg_compat/extended.txt | 10 +- .../q77a.native_datafusion/extended.txt | 14 +- .../q77a.native_iceberg_compat/extended.txt | 14 +- .../q78.native_datafusion/extended.txt | 6 +- .../q78.native_iceberg_compat/extended.txt | 6 +- .../q80a.native_datafusion/extended.txt | 14 +- .../q80a.native_iceberg_compat/extended.txt | 14 +- .../q86a.native_datafusion/extended.txt | 14 +- .../q86a.native_iceberg_compat/extended.txt | 14 +- .../q98.native_datafusion/extended.txt | 6 +- .../q98.native_iceberg_compat/extended.txt | 6 +- .../apache/comet/exec/CometExecSuite.scala | 80 ++++++++- 656 files changed, 3595 insertions(+), 3526 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index c21c0c00e0..1dac4b3896 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -319,7 +319,10 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { * CometBroadcastExchange, enabling broadcast reuse. */ private def transformSubqueryBroadcasts(plan: SparkPlan): SparkPlan = { - plan.transformUpWithSubqueries { case p => + // Use transformUp instead of transformUpWithSubqueries to avoid breaking ReusedSubqueryExec + // references. InSubqueryExec (for DPP) only appears at the top level of filter expressions + // on scans, not nested inside scalar subqueries, so we don't need to recurse into subqueries. + plan.transformUp { case p => p.transformExpressions { case sub: InSubqueryExec => sub.plan match { case s: SubqueryBroadcastExec => diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt index 7293d262f9..dbb2d0ee0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 49 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt index fb562f1904..2e0765d210 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -58,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 49 eligible operators (38%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt index e69fb2ee81..4f79538f22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,8 +43,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -88,4 +88,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 86 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-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt index 28aac3cb2c..0751ddb792 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/extended.txt @@ -23,8 +23,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -49,8 +49,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -103,4 +103,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 86 eligible operators (34%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt index 087450c911..f0d8979a7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt index 9d5b4e7b02..9203ef1c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt index 227d98e43b..8f443004e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 38 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt index 95764f1c1f..de6190cb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ HashAggregate : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 38 eligible operators (47%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt index 2fefb0e3e3..18fed9f207 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt @@ -48,8 +48,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,8 +70,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -134,8 +134,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -212,8 +212,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -276,8 +276,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -354,8 +354,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -418,8 +418,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -466,4 +466,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 431 out of 458 eligible operators (94%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt index 0a57c2d1dd..d0648ae908 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -56,8 +56,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -81,8 +81,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -157,8 +157,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -249,8 +249,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -325,8 +325,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -417,8 +417,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -493,8 +493,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -550,4 +550,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 171 out of 458 eligible operators (37%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt index 16aa5801f1..aaa49f918e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt @@ -43,8 +43,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery @@ -70,8 +70,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -134,8 +134,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -180,7 +180,11 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter @@ -196,8 +200,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- Subquery @@ -223,8 +227,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -287,8 +291,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -333,7 +337,11 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : +- ReusedSubquery + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 333 eligible operators (93%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt index 8af2e7638a..be7b23fb19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/extended.txt @@ -50,8 +50,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery @@ -80,8 +80,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,8 +156,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -211,11 +211,7 @@ TakeOrderedAndProject : +- CometColumnarToRow : +- CometProject : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Filter @@ -233,8 +229,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- Subquery @@ -263,8 +259,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -339,8 +335,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -394,11 +390,7 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometProject +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 327 eligible operators (39%). Final plan contains 67 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt index 47df1d6e25..9b9a06a2d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt index 4146bedb1e..eefbb7001e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/extended.txt @@ -13,8 +13,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 28 eligible operators (46%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt index b168abc169..4afd5872ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt index d7f321c587..8f0455c2e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 57 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt index 2a0b449e0e..309ffb4b14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt index e39c438002..9f223b4efb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -55,4 +55,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 47 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt index 79803b0fa5..894be474f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt index 8cf12e702d..42a548e116 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt index 4b4521d08b..a0a5de8b0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt index acbb025ee5..94dd36d046 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 11 out of 27 eligible operators (40%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt index e03a649133..326f506d99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,4 +30,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 29 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt index 21c4d589cc..30991170e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 29 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt index cb936c9470..42e23d428e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -61,8 +61,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -111,8 +111,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -143,4 +143,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 138 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt index 4df32e5422..3152f94253 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,8 +34,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -68,8 +68,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -123,8 +123,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -157,4 +157,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 138 eligible operators (63%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt index 750bc39460..4abdbd15ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,8 +34,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,8 +64,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -141,8 +141,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -195,4 +195,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 181 out of 190 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt index 22c017a703..2f2648239f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -37,8 +37,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,8 +71,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -153,8 +153,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -209,4 +209,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 135 out of 190 eligible operators (71%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt index b168abc169..4afd5872ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt index d7f321c587..8f0455c2e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 57 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt index bc4d4fdfb2..6d592440af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt index 0be67a7720..5b1f76f0b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt index 4aba7f0f85..f63a48cb16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt index b587ca369f..46d30a7edd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -42,4 +42,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 36 eligible operators (47%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt index c2f112b9bf..4a809ba54b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,8 +38,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt index 40c163c75c..6201ef4aaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,8 +41,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 61 eligible operators (45%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt index d7fec54ac1..a25fbda325 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt index d3dca72ef8..a0414cdd43 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt index 0e4c97d938..22edfeadb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -40,8 +40,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -61,8 +61,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -123,4 +123,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 114 out of 120 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt index 149e1e531e..a79b4dbe02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange @@ -46,8 +46,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange @@ -71,8 +71,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -147,4 +147,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 120 eligible operators (34%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt index 7a02843f94..c5346ea7ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt @@ -11,8 +11,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt index 06178b8214..6aa71efee9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/extended.txt @@ -12,8 +12,8 @@ HashAggregate : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 38 eligible operators (39%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt index 3755a89ee0..62767c2d8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -94,4 +94,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 93 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt index 07f1d47704..7d9075e3c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,4 +109,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 93 eligible operators (50%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt index 518e9ce811..21726ecfaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt index 6abb66ed9d..ef87e9bf80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -40,4 +40,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 34 eligible operators (47%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt index dd5c2f4349..14b5f81ba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt index 9e7faaf1bd..4917c85eb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 30 eligible operators (53%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt index 3a72b7cb9f..7bac15ddd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt index a9b7240a32..e6b374d572 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -76,4 +76,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 66 eligible operators (54%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt index f28434c7c8..0f0ed23eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt index cc5311b498..2fde39534a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 60 eligible operators (43%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt index f28434c7c8..0f0ed23eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt index cc5311b498..2fde39534a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 60 eligible operators (43%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt index 2b3c9ee512..464e28cb22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -46,8 +46,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -128,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 120 out of 126 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-spark3_5/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt index f7d5beaa77..d5827dd62d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange @@ -52,8 +52,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -151,4 +151,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 126 eligible operators (33%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt index 598f4b63a7..169c40a111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -37,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt index 3a2fb76170..af20f56b11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 36 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-spark3_5/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt index 4b7559eccd..963f8a1f13 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 41 eligible operators (80%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt index 31d2a2406d..d621c6cdad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 41 eligible operators (46%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt index d35968a3da..dd09cc7740 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,4 +46,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt index b9b98bcf7c..707a69c75b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 45 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt index 33a209d851..123608fe62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt index 8eb85dfa14..31652f9631 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt index f39a6f8a31..88d0f8b556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt index 9e232766f1..92e43cfbd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ HashAggregate : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 33 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt index a7caa82ba5..363d0b6885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -91,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt index a3c176a9b1..b597b639a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 87 eligible operators (39%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt index 1abbe590ba..e8d4135a12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt index a2ad1c3fa0..bc33c83c77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 86 eligible operators (33%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt index 07d160c357..becbfc5af9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt index 10ba922624..f24630aedc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt index 3faa5d6b77..84b58aa688 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 47 eligible operators (80%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt index 6e7e156e7f..619b1db33d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt @@ -23,8 +23,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,4 +56,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 47 eligible operators (51%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt index b5161dbc09..edbe8704c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometBroadcastExchange : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -52,8 +52,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : :- Subquery @@ -84,8 +84,22 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery + : :- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 96 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt index 9ba26a58f9..b4330bb41b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/extended.txt @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : : : +- ColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometBroadcastExchange : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : :- Subquery @@ -95,22 +95,8 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometProject +- CometFilter - : :- 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 + : :- ReusedSubquery + : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 51 out of 96 eligible operators (53%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 84 eligible operators (48%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt index 9b0203f98a..aa1423da24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -97,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt index 3682583e54..2e5c6b78d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,4 +112,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 96 eligible operators (52%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt index a6f94c745d..28d660a8ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt index 1405dc9cbd..1fb18a8b3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt index 9110f8f632..948332d301 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -109,4 +109,4 @@ CometColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 96 out of 104 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 104 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt index f6a3e56d6b..b2ed88ed32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -120,4 +120,4 @@ TakeOrderedAndProject : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 104 eligible operators (53%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 104 eligible operators (54%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt index 6c0981f51f..764a849e8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- Subquery @@ -38,7 +38,14 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject @@ -54,4 +61,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 58 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt index abccd11311..05963abfce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- Subquery @@ -41,14 +41,7 @@ TakeOrderedAndProject : +- CometColumnarToRow : +- CometProject : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- CometColumnarToRow @@ -65,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt index 9b0203f98a..aa1423da24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -97,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt index 3682583e54..2e5c6b78d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,4 +112,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 96 eligible operators (52%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt index c54bef9b6f..480ee244e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt @@ -19,8 +19,8 @@ Project : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -85,4 +85,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 83 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt index a58b8a8842..5145093c1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ Project : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,4 +98,4 @@ Project +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 83 eligible operators (44%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt index f1364542ed..67dc79ae17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt @@ -47,8 +47,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -167,8 +167,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject @@ -244,4 +244,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt index ff3f54fc27..8b8a5e9c95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/extended.txt @@ -48,8 +48,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -170,8 +170,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -248,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 230 out of 242 eligible operators (95%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt index 7265aa9514..0f38f5a25d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,4 +49,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt index b6a5955810..5814e4d964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -57,4 +57,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 48 eligible operators (37%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt index 7112fbed2b..6409efe7ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -67,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 66 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt index 0592eff15c..e7faf71398 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -79,4 +79,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt index 5643bbe8e5..2300d11dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 37 eligible operators (83%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt index ff50de5a95..5f5a0a5db2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -43,4 +43,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 15 out of 37 eligible operators (40%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 37 eligible operators (43%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt index d35968a3da..dd09cc7740 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,4 +46,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt index b9b98bcf7c..707a69c75b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 45 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt index 85ed0baa1c..14d2953e5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 53 eligible operators (67%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt index 57d98184f4..a6918dcbc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -63,4 +63,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 53 eligible operators (41%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt index 35091ce142..1a89360c78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt index 180543d784..0cbb42f6b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt index a0ad8fbd3c..cc4ecdb736 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt index ed290f4f9d..9612c8faa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 53 eligible operators (33%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 53 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt index 4dfcc2f8ff..12fe53df44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt index f908086f9d..22483ebdbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 49 eligible operators (44%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt index 5e78f07bd6..c4ed904bae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -69,4 +69,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt index 15f59cd399..6d436727e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/extended.txt @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 68 eligible operators (55%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 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-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt index 1c2b08ba4c..4b942f9ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt index f07b682231..b1dc1eae38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -106,8 +106,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -169,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt index 63679c655c..3f5720e58b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -116,8 +116,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -187,4 +187,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 167 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt index c3675de988..7e93a124c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -114,4 +114,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 109 eligible operators (87%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt index e8260ee91b..d0cdc87430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -132,4 +132,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt index 5b69269c61..ee61e9847b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -77,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt index 996b4bed53..bb7c3f0ba8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -80,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 76 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt index add26d22d8..baaed60edf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt index 9d3f8aea2c..680c233293 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt index 86b85906aa..b66ffb7e48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,4 +49,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 48 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt index b33967c1aa..25041c813c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/extended.txt @@ -13,8 +13,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 48 eligible operators (68%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt index cef9fe5220..56c1bb2580 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -128,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt index df0da2d681..ecf5bf9154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -131,4 +131,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 118 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt index f089c495ef..bc7081e007 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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-spark3_5/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt index 4b0542e8fa..e568a58882 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt index 3fa31193d5..c634fde85e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt index bb9078306f..95324b29ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 30 eligible operators (53%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt index aeecb4336a..c0946d9d53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -102,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt index 96ed4c4678..07b83f4880 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -113,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 60 out of 101 eligible operators (59%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 101 eligible operators (60%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt index dde36ec726..dc5120a92c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 52 eligible operators (98%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt index 59b2bac407..4dc6f61baf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -61,4 +61,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 52 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt index 86dc323961..2a1b474f0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,4 +29,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt index bd28ea016b..6f17d97f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 28 eligible operators (46%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt index dde12be29e..5c6e1f7bbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt @@ -16,8 +16,8 @@ HashAggregate : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,4 +70,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 66 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt index 023435c9e2..31bc2e57ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ HashAggregate : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 66 eligible operators (43%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt index 9f7d7f87c6..bc3c215a3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt index 39a53e3d01..c233b212d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,4 +56,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 47 eligible operators (51%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt index 9e5e99015a..23e9ff06a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt @@ -11,8 +11,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt index 5b4b326eb9..9d6c574291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/extended.txt @@ -12,8 +12,8 @@ HashAggregate : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 38 eligible operators (39%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt index 331f00fe36..429a1dfe70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt index 49c3b29af3..8af4dc1d6b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : :- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 33 eligible operators (63%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt index fc4889c2d2..08b3c7450f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt index 0a3a70f3f9..e490f39415 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 29 eligible operators (55%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt index 7293d262f9..dbb2d0ee0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 49 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/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 fb562f1904..2e0765d210 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 @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -58,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 49 eligible operators (38%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt index e69fb2ee81..4f79538f22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,8 +43,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -88,4 +88,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 86 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/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 28aac3cb2c..0751ddb792 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 @@ -23,8 +23,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -49,8 +49,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -103,4 +103,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 86 eligible operators (34%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt index 087450c911..f0d8979a7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt index 9d5b4e7b02..9203ef1c11 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 @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt index 227d98e43b..8f443004e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 38 eligible operators (97%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt index 95764f1c1f..de6190cb38 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 @@ -16,8 +16,8 @@ HashAggregate : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 38 eligible operators (47%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt index 2fefb0e3e3..18fed9f207 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt @@ -48,8 +48,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,8 +70,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -134,8 +134,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -212,8 +212,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -276,8 +276,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -354,8 +354,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -418,8 +418,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -466,4 +466,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 431 out of 458 eligible operators (94%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt index 0a57c2d1dd..d0648ae908 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/extended.txt @@ -56,8 +56,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -81,8 +81,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -157,8 +157,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -249,8 +249,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -325,8 +325,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -417,8 +417,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -493,8 +493,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -550,4 +550,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 171 out of 458 eligible operators (37%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt index 7260efac0a..397e618400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt @@ -43,8 +43,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery @@ -71,8 +71,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,8 +135,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -183,7 +183,11 @@ CometColumnarToRow : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- ReusedSubquery + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter : +- ReusedSubquery @@ -198,8 +202,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery @@ -226,8 +230,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -290,8 +294,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -338,6 +342,10 @@ CometColumnarToRow +- CometFilter : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim - +- ReusedSubquery + +- Subquery + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 337 eligible operators (91%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt index d6d0b2eba6..7bf65a8e2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt @@ -50,8 +50,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery @@ -81,8 +81,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -157,8 +157,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -214,11 +214,7 @@ TakeOrderedAndProject : +- 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 + : +- ReusedSubquery +- BroadcastExchange +- Filter : +- ReusedSubquery @@ -235,8 +231,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery @@ -266,8 +262,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -342,8 +338,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -399,10 +395,6 @@ TakeOrderedAndProject +- 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 + +- ReusedSubquery -Comet accelerated 128 out of 337 eligible operators (37%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 331 eligible operators (38%). Final plan contains 67 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt index 47df1d6e25..9b9a06a2d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt index 4146bedb1e..eefbb7001e 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 @@ -13,8 +13,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 28 eligible operators (46%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt index b168abc169..4afd5872ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt index d7f321c587..8f0455c2e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 57 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt index 2a0b449e0e..309ffb4b14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 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/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt index e39c438002..9f223b4efb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -55,4 +55,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 47 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt index 79803b0fa5..894be474f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt index 8cf12e702d..42a548e116 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 @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt index 4b4521d08b..a0a5de8b0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt index acbb025ee5..94dd36d046 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 11 out of 27 eligible operators (40%). 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/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt index e03a649133..326f506d99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,4 +30,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 29 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt index 21c4d589cc..30991170e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 29 eligible operators (44%). 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/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt index cb936c9470..42e23d428e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -61,8 +61,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -111,8 +111,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -143,4 +143,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 138 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt index 4df32e5422..3152f94253 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,8 +34,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -68,8 +68,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -123,8 +123,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -157,4 +157,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 138 eligible operators (63%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt index 750bc39460..4abdbd15ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,8 +34,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,8 +64,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -141,8 +141,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -195,4 +195,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 181 out of 190 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt index 22c017a703..2f2648239f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -37,8 +37,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,8 +71,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -153,8 +153,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -209,4 +209,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 135 out of 190 eligible operators (71%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt index b168abc169..4afd5872ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 d7f321c587..8f0455c2e1 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 @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 57 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt index bc4d4fdfb2..6d592440af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 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/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt index 0be67a7720..5b1f76f0b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt index 4aba7f0f85..f63a48cb16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt index b587ca369f..46d30a7edd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -42,4 +42,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 36 eligible operators (47%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt index c2f112b9bf..4a809ba54b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,8 +38,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 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/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt index 40c163c75c..6201ef4aaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,8 +41,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 61 eligible operators (45%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt index d7fec54ac1..a25fbda325 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/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 d3dca72ef8..a0414cdd43 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 @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt index 0e4c97d938..22edfeadb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -40,8 +40,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -61,8 +61,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -123,4 +123,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 114 out of 120 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 149e1e531e..a79b4dbe02 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 @@ -21,8 +21,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange @@ -46,8 +46,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange @@ -71,8 +71,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -147,4 +147,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 120 eligible operators (34%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt index 7a02843f94..c5346ea7ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt @@ -11,8 +11,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/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 06178b8214..6aa71efee9 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 @@ -12,8 +12,8 @@ HashAggregate : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 38 eligible operators (39%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt index 3755a89ee0..62767c2d8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -94,4 +94,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 93 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/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 07f1d47704..7d9075e3c8 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 @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,4 +109,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 93 eligible operators (50%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 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/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt index 518e9ce811..21726ecfaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt index 6abb66ed9d..ef87e9bf80 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 @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -40,4 +40,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 34 eligible operators (47%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt index dd5c2f4349..14b5f81ba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt index 9e7faaf1bd..4917c85eb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 30 eligible operators (53%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt index 3a72b7cb9f..7bac15ddd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 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/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt index a9b7240a32..e6b374d572 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -76,4 +76,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 66 eligible operators (54%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt index f28434c7c8..0f0ed23eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 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/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt index cc5311b498..2fde39534a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 60 eligible operators (43%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt index f28434c7c8..0f0ed23eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt index cc5311b498..2fde39534a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 60 eligible operators (43%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt index 2b3c9ee512..464e28cb22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -46,8 +46,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -128,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 120 out of 126 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/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt index f7d5beaa77..d5827dd62d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange @@ -52,8 +52,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -151,4 +151,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 126 eligible operators (33%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt index 598f4b63a7..169c40a111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -37,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 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/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt index 3a2fb76170..af20f56b11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 36 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/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt index 4b7559eccd..963f8a1f13 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 41 eligible operators (80%). Final plan contains 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/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt index 31d2a2406d..d621c6cdad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 41 eligible operators (46%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt index d35968a3da..dd09cc7740 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,4 +46,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt index b9b98bcf7c..707a69c75b 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 @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 45 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt index 33a209d851..123608fe62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt index 8eb85dfa14..31652f9631 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 @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt index f39a6f8a31..88d0f8b556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 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/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt index 9e232766f1..92e43cfbd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ HashAggregate : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 33 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt index a7caa82ba5..363d0b6885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -91,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt index a3c176a9b1..b597b639a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 87 eligible operators (39%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt index 1abbe590ba..e8d4135a12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 86 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/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 a2ad1c3fa0..bc33c83c77 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 @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 86 eligible operators (33%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt index 07d160c357..becbfc5af9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 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/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt index 10ba922624..f24630aedc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt index 3faa5d6b77..84b58aa688 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 47 eligible operators (80%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt index 6e7e156e7f..619b1db33d 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 @@ -23,8 +23,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,4 +56,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 47 eligible operators (51%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 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 @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt index 798f7eed45..5549dc889a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometBroadcastExchange : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -52,8 +52,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : :- ReusedSubquery @@ -89,7 +89,21 @@ CometColumnarToRow : :- ReusedSubquery : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim - :- ReusedSubquery - +- ReusedSubquery + :- Subquery + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 88 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 100 eligible operators (89%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 924a4e9ae6..7c97301a94 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 @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : : : +- ColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometBroadcastExchange : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : :- ReusedSubquery @@ -100,21 +100,7 @@ TakeOrderedAndProject : :- 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 + :- ReusedSubquery + +- ReusedSubquery -Comet accelerated 51 out of 100 eligible operators (51%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 88 eligible operators (46%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt index 9b0203f98a..aa1423da24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -97,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 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/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 3682583e54..2e5c6b78d2 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 @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,4 +112,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 96 eligible operators (52%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt index a6f94c745d..28d660a8ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt index 1405dc9cbd..1fb18a8b3f 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 @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt index 6a006afae8..351796c4a7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -113,4 +113,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 96 out of 108 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 108 eligible operators (89%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt index 49b87191ca..0d1dad3c68 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 @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -124,4 +124,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 108 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 108 eligible operators (52%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt index bd21dfad52..3641dc35a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery @@ -41,7 +41,14 @@ CometColumnarToRow : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- ReusedSubquery + : +- Subquery + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometBroadcastHashJoin @@ -56,4 +63,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 60 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt index 8f75f8eade..d6ce47582d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery @@ -44,14 +44,7 @@ TakeOrderedAndProject : +- CometFilter : : +- 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 + : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow +- CometProject @@ -67,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 39 out of 60 eligible operators (65%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt index 9b0203f98a..aa1423da24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -97,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 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/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 3682583e54..2e5c6b78d2 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 @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,4 +112,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 96 eligible operators (52%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt index c54bef9b6f..480ee244e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt @@ -19,8 +19,8 @@ Project : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -85,4 +85,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 83 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/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 a58b8a8842..5145093c1a 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 @@ -20,8 +20,8 @@ Project : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,4 +98,4 @@ Project +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 83 eligible operators (44%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 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 @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt index f1364542ed..67dc79ae17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt @@ -47,8 +47,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -167,8 +167,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject @@ -244,4 +244,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt index ff3f54fc27..8b8a5e9c95 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 @@ -48,8 +48,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -170,8 +170,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -248,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 230 out of 242 eligible operators (95%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt index 7265aa9514..0f38f5a25d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,4 +49,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/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 b6a5955810..5814e4d964 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 @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -57,4 +57,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 48 eligible operators (37%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt index 7112fbed2b..6409efe7ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -67,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 66 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/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt index 0592eff15c..e7faf71398 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -79,4 +79,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt index 5643bbe8e5..2300d11dd5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 37 eligible operators (83%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt index ff50de5a95..5f5a0a5db2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -43,4 +43,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 15 out of 37 eligible operators (40%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 37 eligible operators (43%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt index d35968a3da..dd09cc7740 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,4 +46,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt index b9b98bcf7c..707a69c75b 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 @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 45 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt index 85ed0baa1c..14d2953e5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 53 eligible operators (67%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt index 57d98184f4..a6918dcbc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -63,4 +63,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 53 eligible operators (41%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt index 35091ce142..1a89360c78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 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/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt index 180543d784..0cbb42f6b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt index a0ad8fbd3c..cc4ecdb736 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 ed290f4f9d..9612c8faa5 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 @@ -17,8 +17,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 53 eligible operators (33%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 53 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt index 4dfcc2f8ff..12fe53df44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/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 f908086f9d..22483ebdbd 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 @@ -21,8 +21,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 49 eligible operators (44%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt index 5e78f07bd6..c4ed904bae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -69,4 +69,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 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/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt index 15f59cd399..6d436727e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/extended.txt @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 68 eligible operators (55%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 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/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 1c2b08ba4c..4b942f9ef9 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 @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt index f07b682231..b1dc1eae38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -106,8 +106,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -169,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt index 63679c655c..3f5720e58b 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 @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -116,8 +116,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -187,4 +187,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 167 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt index c3675de988..7e93a124c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -114,4 +114,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 109 eligible operators (87%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt index e8260ee91b..d0cdc87430 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 @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -132,4 +132,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt index 5b69269c61..ee61e9847b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -77,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 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/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 996b4bed53..bb7c3f0ba8 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 @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -80,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 76 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt index add26d22d8..baaed60edf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt index 9d3f8aea2c..680c233293 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 @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt index 86b85906aa..b66ffb7e48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,4 +49,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 48 eligible operators (97%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt index b33967c1aa..25041c813c 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 @@ -13,8 +13,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 48 eligible operators (68%). 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/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt index cef9fe5220..56c1bb2580 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -128,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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 df0da2d681..ecf5bf9154 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 @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -131,4 +131,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 118 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt index f089c495ef..bc7081e007 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/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 4b0542e8fa..e568a58882 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 @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt index 3fa31193d5..c634fde85e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 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/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt index bb9078306f..95324b29ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 30 eligible operators (53%). 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/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt index aeecb4336a..c0946d9d53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -102,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt index 96ed4c4678..07b83f4880 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -113,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 60 out of 101 eligible operators (59%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 101 eligible operators (60%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt index dde36ec726..dc5120a92c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 52 eligible operators (98%). Final plan contains 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/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt index 59b2bac407..4dc6f61baf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -61,4 +61,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 52 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt index 86dc323961..2a1b474f0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,4 +29,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt index bd28ea016b..6f17d97f1e 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 @@ -17,8 +17,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 28 eligible operators (46%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt index dde12be29e..5c6e1f7bbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt @@ -16,8 +16,8 @@ HashAggregate : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,4 +70,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 66 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt index 023435c9e2..31bc2e57ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ HashAggregate : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 66 eligible operators (43%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 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 @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt index 9f7d7f87c6..bc3c215a3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt index 39a53e3d01..c233b212d5 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 @@ -26,8 +26,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,4 +56,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 47 eligible operators (51%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt index 9e5e99015a..23e9ff06a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt @@ -11,8 +11,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/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 5b4b326eb9..9d6c574291 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 @@ -12,8 +12,8 @@ HashAggregate : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 38 eligible operators (39%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt index 331f00fe36..429a1dfe70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt index 49c3b29af3..8af4dc1d6b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : :- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 33 eligible operators (63%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt index fc4889c2d2..08b3c7450f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt index 0a3a70f3f9..e490f39415 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 @@ -19,8 +19,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 29 eligible operators (55%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt index 7293d262f9..dbb2d0ee0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 49 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/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt index fb562f1904..2e0765d210 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -58,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 49 eligible operators (38%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt index e69fb2ee81..4f79538f22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -43,8 +43,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -88,4 +88,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 86 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/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt index 28aac3cb2c..0751ddb792 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt @@ -23,8 +23,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -49,8 +49,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -103,4 +103,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 86 eligible operators (34%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt index 087450c911..f0d8979a7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 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/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt index 9d5b4e7b02..9203ef1c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt index 227d98e43b..8f443004e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 38 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt index 95764f1c1f..de6190cb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ HashAggregate : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 38 eligible operators (47%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index 2fefb0e3e3..18fed9f207 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt @@ -48,8 +48,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,8 +70,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -134,8 +134,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -212,8 +212,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -276,8 +276,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -354,8 +354,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -418,8 +418,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -466,4 +466,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 431 out of 458 eligible operators (94%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt index 0a57c2d1dd..d0648ae908 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt @@ -56,8 +56,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -81,8 +81,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -157,8 +157,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -249,8 +249,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -325,8 +325,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -417,8 +417,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -493,8 +493,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -550,4 +550,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 171 out of 458 eligible operators (37%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt index 16aa5801f1..aaa49f918e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt @@ -43,8 +43,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery @@ -70,8 +70,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -134,8 +134,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -180,7 +180,11 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter @@ -196,8 +200,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- Subquery @@ -223,8 +227,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -287,8 +291,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -333,7 +337,11 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : +- ReusedSubquery + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 333 eligible operators (93%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt index 8af2e7638a..be7b23fb19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt @@ -50,8 +50,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery @@ -80,8 +80,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,8 +156,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -211,11 +211,7 @@ TakeOrderedAndProject : +- CometColumnarToRow : +- CometProject : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Filter @@ -233,8 +229,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- Subquery @@ -263,8 +259,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -339,8 +335,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -394,11 +390,7 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometProject +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 327 eligible operators (39%). Final plan contains 67 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt index 47df1d6e25..9b9a06a2d3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 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/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt index 4146bedb1e..eefbb7001e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt @@ -13,8 +13,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 28 eligible operators (46%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt index b168abc169..4afd5872ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt index d7f321c587..8f0455c2e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 57 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt index 2a0b449e0e..309ffb4b14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 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/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt index e39c438002..9f223b4efb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -55,4 +55,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 47 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt index 79803b0fa5..894be474f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt index 8cf12e702d..42a548e116 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt index 4b4521d08b..a0a5de8b0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt index acbb025ee5..94dd36d046 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 11 out of 27 eligible operators (40%). 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/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt index e03a649133..326f506d99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,4 +30,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 29 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt index 21c4d589cc..30991170e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 29 eligible operators (44%). 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/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt index cb936c9470..42e23d428e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -61,8 +61,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -111,8 +111,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -143,4 +143,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 138 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt index 4df32e5422..3152f94253 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,8 +34,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -68,8 +68,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -123,8 +123,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -157,4 +157,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 138 eligible operators (63%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt index 750bc39460..4abdbd15ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,8 +34,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,8 +64,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -141,8 +141,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -195,4 +195,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 181 out of 190 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt index 22c017a703..2f2648239f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -37,8 +37,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -71,8 +71,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -153,8 +153,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -209,4 +209,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 135 out of 190 eligible operators (71%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index b168abc169..4afd5872ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt index d7f321c587..8f0455c2e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 57 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt index bc4d4fdfb2..6d592440af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 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/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt index 0be67a7720..5b1f76f0b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt index 4aba7f0f85..f63a48cb16 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt index b587ca369f..46d30a7edd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -42,4 +42,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 36 eligible operators (47%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt index c2f112b9bf..4a809ba54b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,8 +38,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 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/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt index 40c163c75c..6201ef4aaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,8 +41,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 61 eligible operators (45%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt index d7fec54ac1..a25fbda325 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt index d3dca72ef8..a0414cdd43 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index 0e4c97d938..22edfeadb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange @@ -40,8 +40,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -61,8 +61,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -123,4 +123,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 114 out of 120 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt index 149e1e531e..a79b4dbe02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : +- BroadcastExchange @@ -46,8 +46,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange @@ -71,8 +71,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -147,4 +147,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 120 eligible operators (34%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt index 7a02843f94..c5346ea7ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt @@ -11,8 +11,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt index 06178b8214..6aa71efee9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt @@ -12,8 +12,8 @@ HashAggregate : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 38 eligible operators (39%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt index 3755a89ee0..62767c2d8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -94,4 +94,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 93 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/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt index 07f1d47704..7d9075e3c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -109,4 +109,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 93 eligible operators (50%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 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/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt index 518e9ce811..21726ecfaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 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/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt index 6abb66ed9d..ef87e9bf80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -40,4 +40,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 34 eligible operators (47%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt index dd5c2f4349..14b5f81ba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt index 9e7faaf1bd..4917c85eb2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 30 eligible operators (53%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt index 3a72b7cb9f..7bac15ddd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -67,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 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/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt index a9b7240a32..e6b374d572 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -76,4 +76,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 66 eligible operators (54%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt index f28434c7c8..0f0ed23eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 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/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt index cc5311b498..2fde39534a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 60 eligible operators (43%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt index f28434c7c8..0f0ed23eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt index cc5311b498..2fde39534a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 60 eligible operators (43%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index 2b3c9ee512..464e28cb22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange @@ -46,8 +46,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -128,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 120 out of 126 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/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt index f7d5beaa77..d5827dd62d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : +- BroadcastExchange @@ -52,8 +52,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -151,4 +151,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 126 eligible operators (33%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt index 598f4b63a7..169c40a111 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -37,4 +37,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 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/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt index 3a2fb76170..af20f56b11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 36 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/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt index 4b7559eccd..963f8a1f13 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 41 eligible operators (80%). Final plan contains 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/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt index 31d2a2406d..d621c6cdad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 41 eligible operators (46%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt index d35968a3da..dd09cc7740 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,4 +46,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 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/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt index b9b98bcf7c..707a69c75b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 45 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt index 33a209d851..123608fe62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 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/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt index 8eb85dfa14..31652f9631 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt index f39a6f8a31..88d0f8b556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 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/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt index 9e232766f1..92e43cfbd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ HashAggregate : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 33 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index a7caa82ba5..363d0b6885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -91,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt index a3c176a9b1..b597b639a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 87 eligible operators (39%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt index 1abbe590ba..e8d4135a12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 86 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/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt index a2ad1c3fa0..bc33c83c77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 86 eligible operators (33%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt index 07d160c357..becbfc5af9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 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/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt index 10ba922624..f24630aedc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index 3faa5d6b77..84b58aa688 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 47 eligible operators (80%). 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/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt index 6e7e156e7f..619b1db33d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt @@ -23,8 +23,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,4 +56,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 47 eligible operators (51%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 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/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt index b5161dbc09..edbe8704c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometBroadcastExchange : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -52,8 +52,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : :- Subquery @@ -84,8 +84,22 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery + : :- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 96 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt index 9ba26a58f9..b4330bb41b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt @@ -31,8 +31,8 @@ TakeOrderedAndProject : : : : : : : : +- ColumnarToRow : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometColumnarToRow + : : : : : : : : +- CometBroadcastExchange : : : : : : : : +- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : :- Subquery @@ -95,22 +95,8 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometProject +- CometFilter - : :- 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 + : :- ReusedSubquery + : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 51 out of 96 eligible operators (53%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 84 eligible operators (48%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt index 9b0203f98a..aa1423da24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -97,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 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/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt index 3682583e54..2e5c6b78d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,4 +112,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 96 eligible operators (52%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt index a6f94c745d..28d660a8ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 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/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt index 1405dc9cbd..1fb18a8b3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt index de4439657e..ce0b00910a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -60,8 +60,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -112,4 +112,4 @@ CometColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 98 out of 108 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 108 eligible operators (92%). Final plan contains 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/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt index 43a73c6636..4107e5ecd9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -65,8 +65,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -123,4 +123,4 @@ TakeOrderedAndProject : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 58 out of 108 eligible operators (53%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 60 out of 108 eligible operators (55%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt index 6c0981f51f..764a849e8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- Subquery @@ -38,7 +38,14 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject @@ -54,4 +61,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 58 eligible operators (94%). Final plan contains 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/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt index abccd11311..05963abfce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- Subquery @@ -41,14 +41,7 @@ TakeOrderedAndProject : +- CometColumnarToRow : +- CometProject : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- CometColumnarToRow @@ -65,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index 9b0203f98a..aa1423da24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -97,4 +97,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 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/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt index 3682583e54..2e5c6b78d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -112,4 +112,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 96 eligible operators (52%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index c54bef9b6f..480ee244e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt @@ -19,8 +19,8 @@ Project : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -85,4 +85,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 83 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/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt index a58b8a8842..5145093c1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ Project : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,4 +98,4 @@ Project +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 83 eligible operators (44%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt index f1364542ed..67dc79ae17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt @@ -47,8 +47,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -167,8 +167,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject @@ -244,4 +244,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 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/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt index ff3f54fc27..8b8a5e9c95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt @@ -48,8 +48,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -170,8 +170,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -248,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 230 out of 242 eligible operators (95%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt index 7265aa9514..0f38f5a25d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,4 +49,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt index b6a5955810..5814e4d964 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -57,4 +57,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 48 eligible operators (37%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index 7112fbed2b..6409efe7ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -67,4 +67,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 66 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/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt index 0592eff15c..e7faf71398 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -79,4 +79,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt index c7d118c3d3..dd660beb09 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 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/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt index 9985a95a0e..e3e8885838 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -40,4 +40,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 34 eligible operators (47%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt index d35968a3da..dd09cc7740 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,4 +46,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt index b9b98bcf7c..707a69c75b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 45 eligible operators (46%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index 85ed0baa1c..14d2953e5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 53 eligible operators (67%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt index 57d98184f4..a6918dcbc6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -63,4 +63,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 53 eligible operators (41%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt index 35091ce142..1a89360c78 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 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/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt index 180543d784..0cbb42f6b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt index 73d2c0bda8..0480c463e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -57,4 +57,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 52 eligible operators (67%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt index 586f97186a..21119ae432 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -61,4 +61,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 52 eligible operators (34%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 52 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt index 4dfcc2f8ff..12fe53df44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -50,4 +50,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt index f908086f9d..22483ebdbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -59,4 +59,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 49 eligible operators (44%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt index 5e78f07bd6..c4ed904bae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -69,4 +69,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 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/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt index 15f59cd399..6d436727e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 68 eligible operators (55%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 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/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt index 1c2b08ba4c..4b942f9ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt index f07b682231..b1dc1eae38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -106,8 +106,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -169,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). 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/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt index 63679c655c..3f5720e58b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -116,8 +116,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -187,4 +187,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 167 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt index c3675de988..7e93a124c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -114,4 +114,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 109 eligible operators (87%). 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/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt index e8260ee91b..d0cdc87430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -132,4 +132,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt index 5b69269c61..ee61e9847b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -77,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 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/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt index 996b4bed53..bb7c3f0ba8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -80,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 76 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt index add26d22d8..baaed60edf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 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/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt index 9d3f8aea2c..680c233293 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -41,4 +41,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 17 out of 35 eligible operators (48%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt index 86b85906aa..b66ffb7e48 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,4 +49,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 48 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt index b33967c1aa..25041c813c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt @@ -13,8 +13,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 48 eligible operators (68%). 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/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt index cef9fe5220..56c1bb2580 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -128,4 +128,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt index df0da2d681..ecf5bf9154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -131,4 +131,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 118 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt index f089c495ef..bc7081e007 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt index 4b0542e8fa..e568a58882 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -72,4 +72,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt index 3fa31193d5..c634fde85e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 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/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt index bb9078306f..95324b29ea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 30 eligible operators (53%). 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/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index 7e841ac579..41e2401b44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt @@ -14,8 +14,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -56,8 +56,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -116,4 +116,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 109 out of 114 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 114 eligible operators (97%). 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/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt index da2d4ae94d..a7094c9100 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -61,8 +61,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -127,4 +127,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 114 eligible operators (63%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 114 eligible operators (64%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt index dde36ec726..dc5120a92c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt @@ -21,8 +21,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 52 eligible operators (98%). Final plan contains 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/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt index 59b2bac407..4dc6f61baf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -61,4 +61,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 52 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt index 86dc323961..2a1b474f0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt @@ -15,8 +15,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -29,4 +29,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt index bd28ea016b..6f17d97f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 28 eligible operators (46%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index dde12be29e..5c6e1f7bbd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt @@ -16,8 +16,8 @@ HashAggregate : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,4 +70,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 66 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt index 023435c9e2..31bc2e57ae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt @@ -17,8 +17,8 @@ HashAggregate : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 66 eligible operators (43%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt index 6ceb59316c..895ff59932 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 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/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt index 27ad1743da..c267894994 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt index 9f7d7f87c6..bc3c215a3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -48,4 +48,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt index 39a53e3d01..c233b212d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -56,4 +56,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 47 eligible operators (51%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt index 9e5e99015a..23e9ff06a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt @@ -11,8 +11,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,4 +39,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt index 5b4b326eb9..9d6c574291 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt @@ -12,8 +12,8 @@ HashAggregate : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -45,4 +45,4 @@ HashAggregate +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 38 eligible operators (39%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt index 331f00fe36..429a1dfe70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt @@ -12,8 +12,8 @@ CometColumnarToRow : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt index 49c3b29af3..8af4dc1d6b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt @@ -13,8 +13,8 @@ CometColumnarToRow : :- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 33 eligible operators (63%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt index fc4889c2d2..08b3c7450f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt index 0a3a70f3f9..e490f39415 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 16 out of 29 eligible operators (55%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt index f6f71ccf92..37c17a36f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt index bdeced7e55..5649244c42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 52 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt index 1c2b08ba4c..4b942f9ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt index 087450c911..f0d8979a7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt index 9d5b4e7b02..9203ef1c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt index 16aa5801f1..aaa49f918e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt @@ -43,8 +43,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery @@ -70,8 +70,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -134,8 +134,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -180,7 +180,11 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter @@ -196,8 +200,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- Subquery @@ -223,8 +227,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -287,8 +291,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -333,7 +337,11 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : +- ReusedSubquery + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 333 eligible operators (93%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt index 8af2e7638a..be7b23fb19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/extended.txt @@ -50,8 +50,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery @@ -80,8 +80,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,8 +156,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -211,11 +211,7 @@ TakeOrderedAndProject : +- CometColumnarToRow : +- CometProject : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Filter @@ -233,8 +229,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- Subquery @@ -263,8 +259,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -339,8 +335,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -394,11 +390,7 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometProject +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 327 eligible operators (39%). Final plan contains 67 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt index a48a4f02cb..a1bdc227dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt @@ -27,8 +27,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -55,8 +55,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -77,8 +77,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -141,8 +141,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -218,8 +218,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -282,8 +282,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -359,8 +359,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -423,8 +423,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -496,8 +496,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -524,8 +524,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -546,8 +546,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -610,8 +610,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -687,8 +687,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -751,8 +751,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -828,8 +828,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -892,8 +892,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -965,8 +965,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -993,8 +993,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1015,8 +1015,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1079,8 +1079,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1156,8 +1156,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1220,8 +1220,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1297,8 +1297,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1361,8 +1361,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1434,8 +1434,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1462,8 +1462,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1484,8 +1484,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1548,8 +1548,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1625,8 +1625,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1689,8 +1689,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1766,8 +1766,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1830,8 +1830,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1903,8 +1903,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1931,8 +1931,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1953,8 +1953,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2017,8 +2017,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2094,8 +2094,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2158,8 +2158,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2235,8 +2235,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2299,8 +2299,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2347,4 +2347,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2167 out of 2302 eligible operators (94%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt index ccbb4fe9da..05c03e27cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/extended.txt @@ -31,8 +31,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,8 +64,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -89,8 +89,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -165,8 +165,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -256,8 +256,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -332,8 +332,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -423,8 +423,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -499,8 +499,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -586,8 +586,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -619,8 +619,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -644,8 +644,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -720,8 +720,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -811,8 +811,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -887,8 +887,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -978,8 +978,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1054,8 +1054,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1141,8 +1141,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1174,8 +1174,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1199,8 +1199,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1275,8 +1275,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1366,8 +1366,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1442,8 +1442,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1533,8 +1533,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1609,8 +1609,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1696,8 +1696,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1729,8 +1729,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1754,8 +1754,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1830,8 +1830,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1921,8 +1921,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1997,8 +1997,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2088,8 +2088,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2164,8 +2164,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2251,8 +2251,8 @@ CometColumnarToRow : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2284,8 +2284,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2309,8 +2309,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2385,8 +2385,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2476,8 +2476,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2552,8 +2552,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2643,8 +2643,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2719,8 +2719,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2776,4 +2776,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 882 out of 2302 eligible operators (38%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt index 0c126ae714..7d837309dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -211,4 +211,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 205 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt index 6909da9aa4..955f79450b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -250,4 +250,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 210 eligible operators (44%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt index 79803b0fa5..894be474f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt index 8cf12e702d..42a548e116 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt index 180582c6a9..2bde373dbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 28 eligible operators (71%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt index 614af2a9b6..33e7ab4bde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt index 3a2d2bd7db..ef2a0b1d65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -77,8 +77,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -108,8 +108,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -139,8 +139,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -156,4 +156,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 146 out of 151 eligible operators (96%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt index 702314cced..cff1513d9c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -90,8 +90,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -127,8 +127,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -164,8 +164,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -184,4 +184,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 69 out of 151 eligible operators (45%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt index 68e31b1eeb..a15e4a5c8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -96,4 +96,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 95 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-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt index 1378dbd839..350fd60d70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -113,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 95 eligible operators (44%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt index f6f71ccf92..37c17a36f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt index bdeced7e55..5649244c42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 52 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt index 195d72cb2b..3f4222c365 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -52,8 +52,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,8 +84,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt index fc61a5c7eb..846766abb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,8 +98,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -119,4 +119,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 99 eligible operators (48%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt index 33a209d851..123608fe62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt index 8eb85dfa14..31652f9631 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt index a7caa82ba5..363d0b6885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -91,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt index a3c176a9b1..b597b639a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 87 eligible operators (39%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt index d76150bc44..0d50e1e4c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt @@ -32,8 +32,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -55,8 +55,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -137,8 +137,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,8 +160,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -213,4 +213,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 142 out of 196 eligible operators (72%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt index ebbf4c78f7..c0ff09edf0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt @@ -34,8 +34,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -151,8 +151,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -177,8 +177,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -237,4 +237,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 86 out of 196 eligible operators (43%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt index a6f94c745d..28d660a8ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt index 1405dc9cbd..1fb18a8b3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt index 1621fcebdd..3e2941b1a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -108,8 +108,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -196,8 +196,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -266,4 +266,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 245 out of 263 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt index b028a77978..1314c516af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -129,8 +129,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -235,8 +235,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -319,4 +319,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 263 eligible operators (34%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt index 6c0981f51f..764a849e8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- Subquery @@ -38,7 +38,14 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject @@ -54,4 +61,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 58 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt index abccd11311..05963abfce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- Subquery @@ -41,14 +41,7 @@ TakeOrderedAndProject : +- CometColumnarToRow : +- CometProject : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- CometColumnarToRow @@ -65,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt index f1364542ed..67dc79ae17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt @@ -47,8 +47,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -167,8 +167,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject @@ -244,4 +244,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt index ff3f54fc27..8b8a5e9c95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/extended.txt @@ -48,8 +48,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -170,8 +170,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -248,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 230 out of 242 eligible operators (95%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt index 87a6683816..399fa124fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -85,8 +85,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -117,8 +117,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -149,8 +149,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -181,8 +181,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -213,8 +213,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -245,8 +245,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -277,8 +277,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -295,4 +295,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 271 out of 285 eligible operators (95%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt index f37b1a2e9e..2464f64b63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,8 +98,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -136,8 +136,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -174,8 +174,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -212,8 +212,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -250,8 +250,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,8 +288,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -326,8 +326,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -347,4 +347,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 127 out of 285 eligible operators (44%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 136 out of 285 eligible operators (47%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt index 34f65c79d0..332241d809 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -76,8 +76,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -132,8 +132,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -173,4 +173,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 105 out of 156 eligible operators (67%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index 1b160f655d..1a87c62702 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -80,8 +80,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,8 +140,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -185,4 +185,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 156 eligible operators (34%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 156 eligible operators (36%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt index 5e78f07bd6..c4ed904bae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -69,4 +69,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt index 15f59cd399..6d436727e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/extended.txt @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 68 eligible operators (55%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt index 1c2b08ba4c..4b942f9ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt index f07b682231..b1dc1eae38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -106,8 +106,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -169,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt index 63679c655c..3f5720e58b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -116,8 +116,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -187,4 +187,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 167 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt index df51f4120c..bb065d899c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -139,8 +139,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -256,8 +256,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -352,4 +352,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 290 out of 332 eligible operators (87%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt index 54f4396187..7a365cc1ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -158,8 +158,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -293,8 +293,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -406,4 +406,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 116 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt index 5b69269c61..ee61e9847b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -77,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 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-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt index 996b4bed53..bb7c3f0ba8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -80,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 76 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-v2_7-spark3_5/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt index f56f66b4dd..380463e662 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -155,8 +155,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -284,8 +284,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -389,4 +389,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 377 out of 386 eligible operators (97%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt index 86bf4d73b8..199b39ef10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/extended.txt @@ -27,8 +27,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -159,8 +159,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -291,8 +291,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -398,4 +398,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 359 out of 386 eligible operators (93%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt index 4374f0aea2..1e7dfa296d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -44,8 +44,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,8 +70,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,4 +84,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt index 41710f125f..c32bddd0d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -51,8 +51,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -82,8 +82,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,4 +98,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 81 eligible operators (48%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt index ca5d077909..70a8e3cad3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,4 +30,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt index 1894bb8648..fb52c44522 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 28 eligible operators (53%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt index f6f71ccf92..37c17a36f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt index bdeced7e55..5649244c42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 52 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 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/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 1c2b08ba4c..4b942f9ef9 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 @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). 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/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt index 087450c911..f0d8979a7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt index 9d5b4e7b02..9203ef1c11 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 @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt index 7260efac0a..397e618400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt @@ -43,8 +43,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery @@ -71,8 +71,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,8 +135,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -183,7 +183,11 @@ CometColumnarToRow : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- ReusedSubquery + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter : +- ReusedSubquery @@ -198,8 +202,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery @@ -226,8 +230,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -290,8 +294,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -338,6 +342,10 @@ CometColumnarToRow +- CometFilter : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim - +- ReusedSubquery + +- Subquery + +- CometColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 337 eligible operators (91%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt index d6d0b2eba6..7bf65a8e2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt @@ -50,8 +50,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- ReusedSubquery @@ -81,8 +81,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -157,8 +157,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -214,11 +214,7 @@ TakeOrderedAndProject : +- 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 + : +- ReusedSubquery +- BroadcastExchange +- Filter : +- ReusedSubquery @@ -235,8 +231,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- ReusedSubquery @@ -266,8 +262,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -342,8 +338,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -399,10 +395,6 @@ TakeOrderedAndProject +- 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 + +- ReusedSubquery -Comet accelerated 128 out of 337 eligible operators (37%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 331 eligible operators (38%). Final plan contains 67 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt index a48a4f02cb..a1bdc227dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt @@ -27,8 +27,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -55,8 +55,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -77,8 +77,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -141,8 +141,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -218,8 +218,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -282,8 +282,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -359,8 +359,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -423,8 +423,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -496,8 +496,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -524,8 +524,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -546,8 +546,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -610,8 +610,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -687,8 +687,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -751,8 +751,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -828,8 +828,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -892,8 +892,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -965,8 +965,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -993,8 +993,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1015,8 +1015,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1079,8 +1079,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1156,8 +1156,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1220,8 +1220,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1297,8 +1297,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1361,8 +1361,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1434,8 +1434,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1462,8 +1462,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1484,8 +1484,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1548,8 +1548,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1625,8 +1625,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1689,8 +1689,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1766,8 +1766,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1830,8 +1830,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1903,8 +1903,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1931,8 +1931,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1953,8 +1953,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2017,8 +2017,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2094,8 +2094,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2158,8 +2158,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2235,8 +2235,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2299,8 +2299,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2347,4 +2347,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2167 out of 2302 eligible operators (94%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt index ccbb4fe9da..05c03e27cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/extended.txt @@ -31,8 +31,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,8 +64,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -89,8 +89,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -165,8 +165,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -256,8 +256,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -332,8 +332,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -423,8 +423,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -499,8 +499,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -586,8 +586,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -619,8 +619,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -644,8 +644,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -720,8 +720,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -811,8 +811,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -887,8 +887,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -978,8 +978,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1054,8 +1054,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1141,8 +1141,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1174,8 +1174,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1199,8 +1199,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1275,8 +1275,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1366,8 +1366,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1442,8 +1442,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1533,8 +1533,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1609,8 +1609,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1696,8 +1696,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1729,8 +1729,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1754,8 +1754,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1830,8 +1830,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1921,8 +1921,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1997,8 +1997,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2088,8 +2088,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2164,8 +2164,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2251,8 +2251,8 @@ CometColumnarToRow : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2284,8 +2284,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2309,8 +2309,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2385,8 +2385,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2476,8 +2476,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2552,8 +2552,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2643,8 +2643,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2719,8 +2719,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2776,4 +2776,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 882 out of 2302 eligible operators (38%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt index 0c126ae714..7d837309dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -211,4 +211,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 205 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt index 6909da9aa4..955f79450b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -250,4 +250,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 210 eligible operators (44%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt index 79803b0fa5..894be474f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt index 8cf12e702d..42a548e116 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 @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt index 180582c6a9..2bde373dbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 28 eligible operators (71%). Final plan contains 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/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt index 614af2a9b6..33e7ab4bde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt index 3a2d2bd7db..ef2a0b1d65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -77,8 +77,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -108,8 +108,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -139,8 +139,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -156,4 +156,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 146 out of 151 eligible operators (96%). Final plan contains 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/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt index 702314cced..cff1513d9c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -90,8 +90,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -127,8 +127,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -164,8 +164,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -184,4 +184,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 69 out of 151 eligible operators (45%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt index 68e31b1eeb..a15e4a5c8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -96,4 +96,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 95 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-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt index 1378dbd839..350fd60d70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -113,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 95 eligible operators (44%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 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/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt index f6f71ccf92..37c17a36f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt index bdeced7e55..5649244c42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 52 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt index 195d72cb2b..3f4222c365 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -52,8 +52,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,8 +84,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt index fc61a5c7eb..846766abb9 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 @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,8 +98,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -119,4 +119,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 99 eligible operators (48%). 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/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt index 33a209d851..123608fe62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt index 8eb85dfa14..31652f9631 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 @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). 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/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt index a7caa82ba5..363d0b6885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -91,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt index a3c176a9b1..b597b639a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 87 eligible operators (39%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt index d76150bc44..0d50e1e4c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt @@ -32,8 +32,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -55,8 +55,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -137,8 +137,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,8 +160,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -213,4 +213,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 142 out of 196 eligible operators (72%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt index ebbf4c78f7..c0ff09edf0 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 @@ -34,8 +34,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -151,8 +151,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -177,8 +177,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -237,4 +237,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 86 out of 196 eligible operators (43%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt index a6f94c745d..28d660a8ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt index 1405dc9cbd..1fb18a8b3f 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 @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). 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/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt index 1621fcebdd..3e2941b1a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -108,8 +108,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -196,8 +196,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -266,4 +266,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 245 out of 263 eligible operators (93%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt index b028a77978..1314c516af 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 @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -129,8 +129,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -235,8 +235,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -319,4 +319,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 263 eligible operators (34%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt index bd21dfad52..3641dc35a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery @@ -41,7 +41,14 @@ CometColumnarToRow : +- CometFilter : : +- ReusedSubquery : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- ReusedSubquery + : +- Subquery + : +- CometColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometBroadcastHashJoin @@ -56,4 +63,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 60 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt index 8f75f8eade..d6ce47582d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- ReusedSubquery @@ -44,14 +44,7 @@ TakeOrderedAndProject : +- CometFilter : : +- 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 + : +- ReusedSubquery +- BroadcastExchange +- CometColumnarToRow +- CometProject @@ -67,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 39 out of 60 eligible operators (65%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt index f1364542ed..67dc79ae17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt @@ -47,8 +47,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -167,8 +167,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject @@ -244,4 +244,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt index ff3f54fc27..8b8a5e9c95 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 @@ -48,8 +48,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -170,8 +170,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -248,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 230 out of 242 eligible operators (95%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt index 87a6683816..399fa124fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -85,8 +85,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -117,8 +117,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -149,8 +149,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -181,8 +181,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -213,8 +213,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -245,8 +245,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -277,8 +277,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -295,4 +295,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 271 out of 285 eligible operators (95%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt index f37b1a2e9e..2464f64b63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,8 +98,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -136,8 +136,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -174,8 +174,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -212,8 +212,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -250,8 +250,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -288,8 +288,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -326,8 +326,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -347,4 +347,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 127 out of 285 eligible operators (44%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 136 out of 285 eligible operators (47%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt index 34f65c79d0..332241d809 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -76,8 +76,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -132,8 +132,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -173,4 +173,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 105 out of 156 eligible operators (67%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt index 1b160f655d..1a87c62702 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 @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -80,8 +80,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -140,8 +140,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -185,4 +185,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 156 eligible operators (34%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 156 eligible operators (36%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt index 5e78f07bd6..c4ed904bae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -69,4 +69,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 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/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt index 15f59cd399..6d436727e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/extended.txt @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 68 eligible operators (55%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 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/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 1c2b08ba4c..4b942f9ef9 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 @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). 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/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt index f07b682231..b1dc1eae38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -106,8 +106,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -169,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt index 63679c655c..3f5720e58b 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 @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -116,8 +116,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -187,4 +187,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 167 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt index df51f4120c..bb065d899c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -139,8 +139,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -256,8 +256,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -352,4 +352,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 290 out of 332 eligible operators (87%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt index 54f4396187..7a365cc1ed 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 @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -158,8 +158,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -293,8 +293,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -406,4 +406,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 116 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt index 5b69269c61..ee61e9847b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -77,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 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-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 996b4bed53..bb7c3f0ba8 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 @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -80,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 76 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-v2_7-spark4_0/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt index f56f66b4dd..380463e662 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -155,8 +155,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -284,8 +284,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -389,4 +389,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 377 out of 386 eligible operators (97%). 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt index 86bf4d73b8..199b39ef10 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 @@ -27,8 +27,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -159,8 +159,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -291,8 +291,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -398,4 +398,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 359 out of 386 eligible operators (93%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt index 4374f0aea2..1e7dfa296d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -44,8 +44,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,8 +70,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,4 +84,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt index 41710f125f..c32bddd0d8 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 @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -51,8 +51,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -82,8 +82,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,4 +98,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 81 eligible operators (48%). 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/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt index ca5d077909..70a8e3cad3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,4 +30,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 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/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt index 1894bb8648..fb52c44522 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 @@ -18,8 +18,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 28 eligible operators (53%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt index f6f71ccf92..37c17a36f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt index bdeced7e55..5649244c42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 52 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 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/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt index 1c2b08ba4c..4b942f9ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). 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/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt index 087450c911..f0d8979a7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 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/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt index 9d5b4e7b02..9203ef1c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index 16aa5801f1..aaa49f918e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt @@ -43,8 +43,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery @@ -70,8 +70,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -134,8 +134,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -180,7 +180,11 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter @@ -196,8 +200,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- Subquery @@ -223,8 +227,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -287,8 +291,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -333,7 +337,11 @@ CometColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : +- ReusedSubquery + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 333 eligible operators (93%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt index 8af2e7638a..be7b23fb19 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt @@ -50,8 +50,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : : +- Subquery @@ -80,8 +80,8 @@ TakeOrderedAndProject : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -156,8 +156,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -211,11 +211,7 @@ TakeOrderedAndProject : +- CometColumnarToRow : +- CometProject : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Filter @@ -233,8 +229,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : : +- Subquery @@ -263,8 +259,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -339,8 +335,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -394,11 +390,7 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometProject +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 128 out of 327 eligible operators (39%). Final plan contains 67 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt index a48a4f02cb..a1bdc227dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt @@ -27,8 +27,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -55,8 +55,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -77,8 +77,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -141,8 +141,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -218,8 +218,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -282,8 +282,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -359,8 +359,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -423,8 +423,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -496,8 +496,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -524,8 +524,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -546,8 +546,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -610,8 +610,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -687,8 +687,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -751,8 +751,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -828,8 +828,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -892,8 +892,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -965,8 +965,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -993,8 +993,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1015,8 +1015,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1079,8 +1079,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1156,8 +1156,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1220,8 +1220,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1297,8 +1297,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1361,8 +1361,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1434,8 +1434,8 @@ CometColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1462,8 +1462,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1484,8 +1484,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1548,8 +1548,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1625,8 +1625,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1689,8 +1689,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1766,8 +1766,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1830,8 +1830,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1903,8 +1903,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1931,8 +1931,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -1953,8 +1953,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2017,8 +2017,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2094,8 +2094,8 @@ CometColumnarToRow : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2158,8 +2158,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2235,8 +2235,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2299,8 +2299,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -2347,4 +2347,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2167 out of 2302 eligible operators (94%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt index ccbb4fe9da..05c03e27cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt @@ -31,8 +31,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,8 +64,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -89,8 +89,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -165,8 +165,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -256,8 +256,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -332,8 +332,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -423,8 +423,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -499,8 +499,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -586,8 +586,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -619,8 +619,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -644,8 +644,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -720,8 +720,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -811,8 +811,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -887,8 +887,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -978,8 +978,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1054,8 +1054,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1141,8 +1141,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1174,8 +1174,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1199,8 +1199,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1275,8 +1275,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1366,8 +1366,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1442,8 +1442,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1533,8 +1533,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1609,8 +1609,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1696,8 +1696,8 @@ CometColumnarToRow : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1729,8 +1729,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1754,8 +1754,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1830,8 +1830,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1921,8 +1921,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -1997,8 +1997,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2088,8 +2088,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2164,8 +2164,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2251,8 +2251,8 @@ CometColumnarToRow : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2284,8 +2284,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2309,8 +2309,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2385,8 +2385,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2476,8 +2476,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2552,8 +2552,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2643,8 +2643,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2719,8 +2719,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -2776,4 +2776,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 882 out of 2302 eligible operators (38%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt index 0c126ae714..7d837309dc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt @@ -19,8 +19,8 @@ CometColumnarToRow : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -211,4 +211,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 205 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt index 6909da9aa4..955f79450b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -250,4 +250,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 210 eligible operators (44%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt index 79803b0fa5..894be474f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -28,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 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/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt index 8cf12e702d..42a548e116 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -32,4 +32,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 13 out of 27 eligible operators (48%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt index 180582c6a9..2bde373dbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -31,4 +31,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 28 eligible operators (71%). Final plan contains 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/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt index 614af2a9b6..33e7ab4bde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt @@ -14,8 +14,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -33,4 +33,4 @@ TakeOrderedAndProject +- CometColumnarToRow +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt index 3a2d2bd7db..ef2a0b1d65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -46,8 +46,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -77,8 +77,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -108,8 +108,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -139,8 +139,8 @@ CometColumnarToRow : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -156,4 +156,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 146 out of 151 eligible operators (96%). Final plan contains 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/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt index 702314cced..cff1513d9c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -53,8 +53,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -90,8 +90,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -127,8 +127,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -164,8 +164,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.inventory [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -184,4 +184,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 69 out of 151 eligible operators (45%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt index 68e31b1eeb..a15e4a5c8e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt @@ -15,8 +15,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -96,4 +96,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 95 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-v2_7/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt index 1378dbd839..350fd60d70 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt @@ -16,8 +16,8 @@ TakeOrderedAndProject : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -113,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 95 eligible operators (44%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt index b63d5c384c..41ab29b225 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -38,4 +38,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 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/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt index ba55221147..24818d477c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -44,4 +44,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 37 eligible operators (51%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 4472ddfbbf..75b9276038 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt index f1832f777a..7e0f1dddeb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt @@ -21,8 +21,8 @@ TakeOrderedAndProject : : : : :- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -64,4 +64,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 54 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt index f6f71ccf92..37c17a36f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt @@ -17,8 +17,8 @@ CometColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -53,4 +53,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt index bdeced7e55..5649244c42 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : :- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -62,4 +62,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 52 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt index 195d72cb2b..3f4222c365 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -52,8 +52,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,8 +84,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt index fc61a5c7eb..846766abb9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,8 +98,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -119,4 +119,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 99 eligible operators (48%). 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/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt index 33a209d851..123608fe62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 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/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt index 8eb85dfa14..31652f9631 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). 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/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index a7caa82ba5..363d0b6885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -91,4 +91,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt index a3c176a9b1..b597b639a0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -103,4 +103,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 87 eligible operators (39%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index d76150bc44..0d50e1e4c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -32,8 +32,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -55,8 +55,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -137,8 +137,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,8 +160,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -213,4 +213,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 142 out of 196 eligible operators (72%). 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/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt index ebbf4c78f7..c0ff09edf0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt @@ -34,8 +34,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -60,8 +60,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -151,8 +151,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -177,8 +177,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -237,4 +237,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 86 out of 196 eligible operators (43%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt index a6f94c745d..28d660a8ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt @@ -27,8 +27,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -59,8 +59,8 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -91,8 +91,8 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 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/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt index 1405dc9cbd..1fb18a8b3f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt @@ -30,8 +30,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -67,8 +67,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -104,8 +104,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange @@ -117,4 +117,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 97 eligible operators (40%). 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/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index 1621fcebdd..3e2941b1a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -108,8 +108,8 @@ CometColumnarToRow : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -196,8 +196,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -266,4 +266,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 245 out of 263 eligible operators (93%). 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/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt index b028a77978..1314c516af 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -129,8 +129,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -235,8 +235,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -319,4 +319,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 263 eligible operators (34%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt index 6c0981f51f..764a849e8b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- Subquery @@ -38,7 +38,14 @@ CometColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject @@ -54,4 +61,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 58 eligible operators (94%). Final plan contains 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/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt index abccd11311..05963abfce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : : +- Subquery @@ -41,14 +41,7 @@ TakeOrderedAndProject : +- CometColumnarToRow : +- CometProject : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- CometColumnarToRow @@ -65,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt index f1364542ed..67dc79ae17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt @@ -47,8 +47,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject @@ -167,8 +167,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject @@ -244,4 +244,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt index ff3f54fc27..8b8a5e9c95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt @@ -48,8 +48,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -170,8 +170,8 @@ CometColumnarToRow : : : : : : : : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometColumnarToRow @@ -248,4 +248,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 230 out of 242 eligible operators (95%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt index 2fd6aa960b..3c50404007 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt @@ -17,8 +17,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -49,8 +49,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -81,8 +81,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -113,8 +113,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -145,8 +145,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -177,8 +177,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -209,8 +209,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -241,8 +241,8 @@ TakeOrderedAndProject : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -273,8 +273,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -291,4 +291,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 270 out of 282 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt index fdea4192ad..4e8421c332 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -57,8 +57,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -95,8 +95,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -133,8 +133,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -171,8 +171,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -209,8 +209,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -247,8 +247,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -285,8 +285,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -323,8 +323,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -344,4 +344,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 127 out of 282 eligible operators (45%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 136 out of 282 eligible operators (48%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt index 27ffd38df9..5384ad8616 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -75,8 +75,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -130,8 +130,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -170,4 +170,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 153 eligible operators (66%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 105 out of 153 eligible operators (68%). 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/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt index dec03f15f7..b34cb0a2ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,8 +79,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -138,8 +138,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -182,4 +182,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 153 eligible operators (35%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 153 eligible operators (37%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt index 5e78f07bd6..c4ed904bae 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt @@ -28,8 +28,8 @@ CometColumnarToRow : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -69,4 +69,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 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/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt index 15f59cd399..6d436727e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt @@ -29,8 +29,8 @@ CometColumnarToRow : : : : : : : : : : +- ColumnarToRow : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometColumnarToRow + : : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : : +- CometProject : : : : : : : : : : +- CometFilter : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -79,4 +79,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 68 eligible operators (55%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt index e0ed00033c..42898c5436 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt @@ -20,8 +20,8 @@ CometColumnarToRow : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -42,8 +42,8 @@ CometColumnarToRow : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange @@ -87,4 +87,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 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/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt index 1c2b08ba4c..4b942f9ef9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt @@ -22,8 +22,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -48,8 +48,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange @@ -102,4 +102,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 85 eligible operators (35%). 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/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt index f07b682231..b1dc1eae38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange @@ -106,8 +106,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange @@ -169,4 +169,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). 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/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt index 63679c655c..3f5720e58b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt @@ -24,8 +24,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : : +- BroadcastExchange @@ -116,8 +116,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- BroadcastExchange @@ -187,4 +187,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 113 out of 167 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index df51f4120c..bb065d899c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt @@ -22,8 +22,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -139,8 +139,8 @@ CometColumnarToRow : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -256,8 +256,8 @@ CometColumnarToRow : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -352,4 +352,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 290 out of 332 eligible operators (87%). 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/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt index 54f4396187..7a365cc1ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt @@ -23,8 +23,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -158,8 +158,8 @@ CometColumnarToRow : : : : : +- ColumnarToRow : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow + : : : : : +- CometColumnarToRow + : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -293,8 +293,8 @@ CometColumnarToRow : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -406,4 +406,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 116 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt index 5b69269c61..ee61e9847b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort @@ -77,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 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-v2_7/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt index 996b4bed53..bb7c3f0ba8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt @@ -19,8 +19,8 @@ TakeOrderedAndProject : : : : +- ColumnarToRow : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow + : : : : +- CometColumnarToRow + : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : : +- CometSort @@ -80,4 +80,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 76 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-v2_7/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt index f56f66b4dd..380463e662 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt @@ -26,8 +26,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -155,8 +155,8 @@ CometColumnarToRow : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -284,8 +284,8 @@ CometColumnarToRow : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -389,4 +389,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 377 out of 386 eligible operators (97%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt index 86bf4d73b8..199b39ef10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt @@ -27,8 +27,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -159,8 +159,8 @@ CometColumnarToRow : : : : : : : +- ColumnarToRow : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometColumnarToRow + : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -291,8 +291,8 @@ CometColumnarToRow : : : : : : +- ColumnarToRow : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow + : : : : : : +- CometColumnarToRow + : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -398,4 +398,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 359 out of 386 eligible operators (93%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt index 4374f0aea2..1e7dfa296d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt @@ -18,8 +18,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -44,8 +44,8 @@ TakeOrderedAndProject : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -70,8 +70,8 @@ TakeOrderedAndProject : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,4 +84,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 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/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt index 41710f125f..c32bddd0d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt @@ -20,8 +20,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -51,8 +51,8 @@ TakeOrderedAndProject : : : +- ColumnarToRow : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -82,8 +82,8 @@ TakeOrderedAndProject : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.web_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -98,4 +98,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 81 eligible operators (48%). 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/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt index ca5d077909..70a8e3cad3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt @@ -16,8 +16,8 @@ CometColumnarToRow : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -30,4 +30,4 @@ CometColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 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/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt index 1894bb8648..fb52c44522 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt @@ -18,8 +18,8 @@ CometColumnarToRow : : +- ColumnarToRow : : +- Scan parquet spark_catalog.default.store_sales [COMET: Dynamic Partition Pruning is not supported with spark.comet.scan.impl=native_iceberg_compat.] : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 15 out of 28 eligible operators (53%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 1ae831f20f..93749185ef 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -2528,9 +2528,6 @@ class CometExecSuite extends CometTestBase { df.collect() val plan = df.queryExecution.executedPlan - // scalastyle:off println - println(s"=== Executed Plan ===\n$plan") - // scalastyle:on println // Check that DPP is triggered with SubqueryBroadcast val subqueryBroadcasts = plan.collectWithSubqueries { case s: SubqueryBroadcastExec => @@ -2562,12 +2559,12 @@ class CometExecSuite extends CometTestBase { } assert( hasReuse, - s"CometBroadcastExchange should be reused.\n" + + "CometBroadcastExchange should be reused.\n" + s"SubqueryBroadcast structure: ${s.child.getClass.getSimpleName} -> " + s"${cbe.getClass.getSimpleName}\nFull plan:\n$plan") case other => fail( - s"Expected CometBroadcastExchangeExec under CometColumnarToRowExec, " + + "Expected CometBroadcastExchangeExec under CometColumnarToRowExec, " + s"got ${other.getClass.getSimpleName}\n$plan") } case b: BroadcastExchangeLike => @@ -2577,7 +2574,7 @@ class CometExecSuite extends CometTestBase { } assert( hasReuse, - s"SubqueryBroadcast's BroadcastExchange should have been reused.\n" + + "SubqueryBroadcast's BroadcastExchange should have been reused.\n" + s"SubqueryBroadcast child: ${s.child.getClass.getSimpleName}\n" + s"Full plan:\n$plan") case other => @@ -2590,8 +2587,77 @@ class CometExecSuite extends CometTestBase { } } -} + test("SubqueryBroadcast transformation preserves ReusedSubquery") { + // This test verifies that when we transform SubqueryBroadcastExec to use + // CometBroadcastExchangeExec, we don't break ReusedSubqueryExec references + // for scalar subqueries. This pattern appears in TPC-DS queries like q6. + withTable("fact_dpp", "dim_dpp") { + // Create partitioned fact table - partition on store_id to trigger DPP + spark + .createDataFrame( + Seq((1L, 1, 100), (2L, 1, 200), (3L, 2, 300), (4L, 2, 400), (5L, 3, 500))) + .toDF("id", "store_id", "value") + .write + .partitionBy("store_id") + .format("parquet") + .saveAsTable("fact_dpp") + + // Create dimension table (small, to be broadcast) + spark + .createDataFrame(Seq((1, "NL"), (2, "DE"))) + .toDF("store_id", "country") + .write + .format("parquet") + .saveAsTable("dim_dpp") + + withSQLConf( + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "true", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "true", + "spark.sql.autoBroadcastJoinThreshold" -> "10MB") { + + // Query that triggers: + // 1. DPP with SubqueryBroadcast (join on partition column store_id) + // 2. Scalar subquery that gets reused (max value used twice) + val query = + """SELECT f.id, f.value, + | (SELECT max(value) FROM fact_dpp) as max_val + |FROM fact_dpp f + |JOIN dim_dpp d ON f.store_id = d.store_id + |WHERE d.country = 'DE' + | AND f.value > (SELECT max(value) FROM fact_dpp) / 10 + |""".stripMargin + val df = sql(query) + val plan = df.queryExecution.executedPlan + + // Check for SubqueryBroadcastExec (DPP) + val subqueryBroadcasts = plan.collectWithSubqueries { case s: SubqueryBroadcastExec => s } + + // Verify ReusedSubqueryExec is preserved for scalar subqueries + val reusedSubqueries = plan.collectWithSubqueries { case rs: ReusedSubqueryExec => rs } + val subqueryExecs = plan.collectWithSubqueries { case s: SubqueryExec => s } + + // We should have DPP triggered (SubqueryBroadcastExec present) + // and scalar subquery reuse preserved (ReusedSubqueryExec present) + assert( + subqueryBroadcasts.nonEmpty, + s"Expected SubqueryBroadcastExec for DPP but found none.\nPlan:\n$plan") + + assert( + reusedSubqueries.nonEmpty || subqueryExecs.length <= 1, + "Expected ReusedSubqueryExec to be preserved but found none. " + + s"Found ${subqueryExecs.length} SubqueryExec instead.\nPlan:\n$plan") + + checkSparkAnswer(df) + } + } + } + +} case class BucketedTableTestSpec( bucketSpec: Option[BucketSpec], numPartitions: Int = 10, From fc13972867912f1857c294f0a1c388344efd3229 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Feb 2026 10:46:07 -0500 Subject: [PATCH 35/38] Update DynamicPartitionPruningSuiteBase. --- dev/diffs/3.5.8.diff | 36 ++++++++++++++++++++++++++++-------- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index d6247f7bb2..a9fdc1fb7f 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -410,18 +410,38 @@ index c4fb4fa943c..a04b23870a8 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..665d414c61f 100644 +index f33432ddb6f..44b59ef70b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.plans.ExistenceJoin -+import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometColumnarToRowExec, CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -262,6 +263,12 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -206,6 +207,19 @@ abstract class DynamicPartitionPruningSuiteBase + s.child match { + case _: ReusedExchangeExec => // reuse check ok. + case BroadcastQueryStageExec(_, _: ReusedExchangeExec, _) => // reuse check ok. ++ case c2r: CometColumnarToRowExec => ++ // Comet wraps CometBroadcastExchangeExec with CometColumnarToRowExec for row output ++ c2r.child match { ++ case b: CometBroadcastExchangeExec => ++ val hasReuse = plan.exists { ++ case ReusedExchangeExec(_, e: CometBroadcastExchangeExec) => ++ e.canonicalized == b.canonicalized ++ case _ => false ++ } ++ assert(hasReuse, s"$s\nshould have been reused in\n$plan") ++ case _ => ++ fail(s"Invalid child node under CometColumnarToRowExec in\n$s") ++ } + case b: BroadcastExchangeLike => + val hasReuse = plan.exists { + case ReusedExchangeExec(_, e) => e eq b +@@ -262,6 +276,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } @@ -434,7 +454,7 @@ index f33432ddb6f..665d414c61f 100644 case _ => Nil } } -@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1047,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -444,7 +464,7 @@ index f33432ddb6f..665d414c61f 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1223,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1236,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -454,7 +474,7 @@ index f33432ddb6f..665d414c61f 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1432,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1445,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -464,7 +484,7 @@ index f33432ddb6f..665d414c61f 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1708,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1721,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -474,7 +494,7 @@ index f33432ddb6f..665d414c61f 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1740,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1753,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) From 47b80c9367e6bba6baf340dd6cbfa05390cffa90 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Feb 2026 12:39:29 -0500 Subject: [PATCH 36/38] Update diff for ReusedExchangeExec wrapping CometBroadcastExchangeExec --- dev/diffs/3.5.8.diff | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index a9fdc1fb7f..3738d40354 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -410,7 +410,7 @@ index c4fb4fa943c..a04b23870a8 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..44b59ef70b2 100644 +index f33432ddb6f..4d2db5000bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -421,7 +421,7 @@ index f33432ddb6f..44b59ef70b2 100644 import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -206,6 +207,19 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -206,6 +207,21 @@ abstract class DynamicPartitionPruningSuiteBase s.child match { case _: ReusedExchangeExec => // reuse check ok. case BroadcastQueryStageExec(_, _: ReusedExchangeExec, _) => // reuse check ok. @@ -435,13 +435,15 @@ index f33432ddb6f..44b59ef70b2 100644 + case _ => false + } + assert(hasReuse, s"$s\nshould have been reused in\n$plan") ++ case ReusedExchangeExec(_, _: CometBroadcastExchangeExec) => ++ // ReusedExchangeExec of CometBroadcastExchangeExec is valid + case _ => + fail(s"Invalid child node under CometColumnarToRowExec in\n$s") + } case b: BroadcastExchangeLike => val hasReuse = plan.exists { case ReusedExchangeExec(_, e) => e eq b -@@ -262,6 +276,12 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -262,6 +278,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } @@ -454,7 +456,7 @@ index f33432ddb6f..44b59ef70b2 100644 case _ => Nil } } -@@ -1027,7 +1047,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1049,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -464,7 +466,7 @@ index f33432ddb6f..44b59ef70b2 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1236,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1238,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -474,7 +476,7 @@ index f33432ddb6f..44b59ef70b2 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1445,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1447,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -484,7 +486,7 @@ index f33432ddb6f..44b59ef70b2 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1721,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1723,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -494,7 +496,7 @@ index f33432ddb6f..44b59ef70b2 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1753,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1755,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) From d885aa3e3b863a9f2d3d9fb16e02ecee37d448de Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Feb 2026 14:56:23 -0500 Subject: [PATCH 37/38] try to do a deferred rule for aqe dpp --- .../scala/org/apache/comet/CometConf.scala | 12 ++ .../comet/CometSparkSessionExtensions.scala | 6 +- .../comet/rules/CometBroadcastJoinRule.scala | 187 ++++++++++++++++++ .../apache/comet/rules/CometExecRule.scala | 31 +++ .../ShimCometSparkSessionExtensions.scala | 9 + .../ShimCometSparkSessionExtensions.scala | 55 ++++++ .../ShimCometSparkSessionExtensions.scala | 10 +- .../apache/comet/exec/CometExecSuite.scala | 88 +++++++++ 8 files changed, 396 insertions(+), 2 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/rules/CometBroadcastJoinRule.scala rename spark/src/main/{spark-3.x => spark-3.4}/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala (83%) create mode 100644 spark/src/main/spark-3.5/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 81e2ce2ee7..89cff2629c 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -149,6 +149,18 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(true) + val COMET_AQE_DPP_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.scan.aqeDpp.enabled") + .category(CATEGORY_SCAN) + .doc( + "Whether to enable AQE-compatible Dynamic Partition Pruning (DPP) for broadcast joins. " + + "When enabled, Comet defers BroadcastHashJoinExec transformation to allow Spark's " + + "PlanAdaptiveDynamicPruningFilters to create DPP, then transforms the join afterward. " + + "When disabled, broadcast joins are transformed immediately which may prevent DPP " + + "from being created in AQE mode. This config only has effect when AQE is enabled.") + .booleanConf + .createWithDefault(true) + val COMET_ICEBERG_NATIVE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.icebergNative.enabled") .category(CATEGORY_SCAN) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 5839570684..5b9495f15f 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.internal.SQLConf import org.apache.comet.CometConf._ -import org.apache.comet.rules.{CometExecRule, CometScanRule, EliminateRedundantTransitions} +import org.apache.comet.rules.{CometBroadcastJoinRule, CometExecRule, CometScanRule, EliminateRedundantTransitions} import org.apache.comet.shims.ShimCometSparkSessionExtensions /** @@ -49,6 +49,10 @@ class CometSparkSessionExtensions extensions.injectColumnar { session => CometExecColumnar(session) } extensions.injectQueryStagePrepRule { session => CometScanRule(session) } extensions.injectQueryStagePrepRule { session => CometExecRule(session) } + // CometBroadcastJoinRule runs AFTER PlanAdaptiveDynamicPruningFilters in AQE mode, + // completing the deferred BroadcastHashJoinExec transformation after DPP is created. + // Uses shim because injectQueryStageOptimizerRule doesn't exist in Spark 3.4. + injectQueryStageOptimizerRules(extensions, session => CometBroadcastJoinRule(session)) } case class CometScanColumnar(session: SparkSession) extends ColumnarRule { diff --git a/spark/src/main/scala/org/apache/comet/rules/CometBroadcastJoinRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometBroadcastJoinRule.scala new file mode 100644 index 0000000000..0fd2164830 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/rules/CometBroadcastJoinRule.scala @@ -0,0 +1,187 @@ +/* + * 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.rules + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.comet._ +import org.apache.spark.sql.execution.{InputAdapter, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec +import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions._ +import org.apache.comet.serde.OperatorOuterClass + +/** + * Rule that transforms BroadcastHashJoinExec to CometBroadcastHashJoinExec in AQE mode. + * + * This rule runs in queryStageOptimizerRules AFTER PlanAdaptiveDynamicPruningFilters, allowing + * DPP to find the broadcast join and create SubqueryBroadcastExec before Comet transforms it. + * + * CometExecRule defers BroadcastHashJoinExec transformation in AQE mode, and this rule completes + * the transformation. + */ +case class CometBroadcastJoinRule(session: SparkSession) extends Rule[SparkPlan] { + + override def apply(plan: SparkPlan): SparkPlan = { + if (!isCometLoaded(conf) || !CometConf.COMET_EXEC_ENABLED.get(conf) || + !CometConf.COMET_AQE_DPP_ENABLED.get(conf) || + CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) != CometConf.SCAN_NATIVE_DATAFUSION) { + return plan + } + + plan.transformUp { + case plan: BroadcastHashJoinExec if hasBroadcastChild(plan) => + transformBroadcastHashJoin(plan) + case other => + other + } + } + + /** Check if plan has a broadcast exchange child (direct or wrapped in QueryStage) */ + private def hasBroadcastChild(plan: SparkPlan): Boolean = { + plan.children.exists { + case _: BroadcastExchangeExec => true + case b: BroadcastQueryStageExec => b.broadcast.isInstanceOf[BroadcastExchangeExec] + case _ => false + } + } + + private def transformBroadcastHashJoin(plan: SparkPlan): SparkPlan = { + val join = plan.asInstanceOf[BroadcastHashJoinExec] + + // Transform children: BroadcastExchangeExec -> CometBroadcastExchangeExec + // For BroadcastQueryStageExec, we need to handle the wrapped exchange + val newChildren = join.children.map { + case b: BroadcastExchangeExec => + convertBroadcastExchange(b).getOrElse(b) + case bqs: BroadcastQueryStageExec => + // In AQE, the broadcast is wrapped in a query stage + // Try to convert the underlying exchange + bqs.broadcast match { + case b: BroadcastExchangeExec => + convertBroadcastExchange(b) match { + case Some(cometExchange) => + // Return a sink placeholder that wraps the query stage + CometSinkPlaceHolder( + cometExchange.asInstanceOf[CometNativeExec].nativeOp, + bqs, + cometExchange) + case None => + bqs + } + case _ => + bqs + } + case other => other + } + + // Check if all children are now native + if (newChildren.forall(_.isInstanceOf[CometNativeExec])) { + val joinWithNewChildren = join.copy(left = newChildren(0), right = newChildren(1)) + convertBroadcastHashJoin(joinWithNewChildren).getOrElse(plan) + } else { + plan + } + } + + private def convertBroadcastExchange(b: BroadcastExchangeExec): Option[SparkPlan] = { + if (!CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED.get(conf)) { + return None + } + + // Check if children are CometNativeExec (unwrap WholeStageCodegen/InputAdapter if needed) + val actualChildren = b.children + .map { + case wsc: WholeStageCodegenExec => wsc.child + case other => other + } + .map { + case ia: InputAdapter => ia.child + case other => other + } + + // Need to find the actual native exec under CometColumnarToRow + val nativeChildren = actualChildren.flatMap { + case c2r: CometColumnarToRowExec => + // Unwrap InputAdapter if present (from WholeStageCodegen) + val actualChild = c2r.child match { + case ia: InputAdapter => ia.child + case other => other + } + Some(actualChild) + case c2r: CometNativeColumnarToRowExec => + val actualChild = c2r.child match { + case ia: InputAdapter => ia.child + case other => other + } + Some(actualChild) + case n: CometNativeExec => + Some(n) + case other => + None + } + + if (nativeChildren.isEmpty || nativeChildren.size != b.children.size) { + return None + } + + // All native children found + if (!nativeChildren.forall(_.isInstanceOf[CometNativeExec])) { + return None + } + + val builder = OperatorOuterClass.Operator.newBuilder().setPlanId(b.id) + val childOp = nativeChildren.map(_.asInstanceOf[CometNativeExec].nativeOp) + childOp.foreach(builder.addChildren) + + // Use the companion object's convert method + val nativeChild = nativeChildren.head + CometBroadcastExchangeExec + .convert(b, builder, childOp: _*) + .map { nativeOp => + CometSinkPlaceHolder( + nativeOp, + b, + CometBroadcastExchangeExec(b, b.output, b.mode, nativeChild)) + } + } + + private def convertBroadcastHashJoin(join: BroadcastHashJoinExec): Option[SparkPlan] = { + if (!CometConf.COMET_EXEC_HASH_JOIN_ENABLED.get(conf)) { + return None + } + + // All children must be CometNativeExec for the join to be converted + if (!join.children.forall(_.isInstanceOf[CometNativeExec])) { + return None + } + + val builder = OperatorOuterClass.Operator.newBuilder().setPlanId(join.id) + val childOps = join.children.map(_.asInstanceOf[CometNativeExec].nativeOp) + childOps.foreach(builder.addChildren) + + CometBroadcastHashJoinExec + .convert(join, builder, childOps: _*) + .map(nativeOp => CometBroadcastHashJoinExec.createExec(nativeOp, join)) + } +} diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 1dac4b3896..01abac20b7 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -225,6 +225,17 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the // broadcast exchange is forced to be enabled by Comet config. case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) => + // In AQE mode, defer BroadcastHashJoinExec transformation to CometBroadcastJoinRule + // ONLY when DPP applies, AQE DPP is enabled, and using native_datafusion scan. + // This allows Spark's PlanAdaptiveDynamicPruningFilters to find the join and create + // SubqueryBroadcastExec. Without deferral, Comet transforms the join before DPP runs. + if (CometConf.COMET_AQE_DPP_ENABLED.get(conf) && + CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) == CometConf.SCAN_NATIVE_DATAFUSION && + conf.adaptiveExecutionEnabled && + plan.isInstanceOf[BroadcastHashJoinExec] && + hasDynamicPruning(plan)) { + return plan + } val newChildren = plan.children.map { case b: BroadcastExchangeExec if b.children.forall(_.isInstanceOf[CometNativeExec]) => convertToComet(b, CometBroadcastExchangeExec).getOrElse(b) @@ -670,4 +681,24 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } } + /** + * Checks if any descendant scan in the plan has dynamic partition pruning expressions. DPP is + * indicated by InSubqueryExec in the scan's partition filters. + */ + private def hasDynamicPruning(plan: SparkPlan): Boolean = { + plan.find { + case s: CometNativeScanExec => + hasInSubqueryExpr(s.partitionFilters) + case s: CometScanExec => + hasInSubqueryExpr(s.partitionFilters) + case f: FileSourceScanExec => + hasInSubqueryExpr(f.partitionFilters) + case _ => false + }.isDefined + } + + private def hasInSubqueryExpr(exprs: Seq[Expression]): Boolean = { + exprs.exists(_.find(_.isInstanceOf[InSubqueryExec]).isDefined) + } + } diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala similarity index 83% rename from spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala rename to spark/src/main/spark-3.4/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala index 0dd783201a..ebeacae3bb 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala +++ b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala @@ -19,6 +19,8 @@ package org.apache.comet.shims +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} trait ShimCometSparkSessionExtensions { @@ -43,4 +45,11 @@ trait ShimCometSparkSessionExtensions { } true } + + // injectQueryStageOptimizerRule doesn't exist in Spark 3.4 + def injectQueryStageOptimizerRules( + extensions: SparkSessionExtensions, + rule: SparkSession => Rule[SparkPlan]): Unit = { + // No-op in Spark 3.4 + } } diff --git a/spark/src/main/spark-3.5/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala new file mode 100644 index 0000000000..33154fb004 --- /dev/null +++ b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala @@ -0,0 +1,55 @@ +/* + * 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.shims + +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} + +trait ShimCometSparkSessionExtensions { + + /** + * TODO: delete after dropping Spark 3.x support and directly call + * SQLConf.EXTENDED_EXPLAIN_PROVIDERS.key + */ + protected val EXTENDED_EXPLAIN_PROVIDERS_KEY = "spark.sql.extendedExplainProviders" + + // Extended info is available only since Spark 4.0.0 + // (https://issues.apache.org/jira/browse/SPARK-47289) + def supportsExtendedExplainInfo(qe: QueryExecution): Boolean = { + try { + // Look for QueryExecution.extendedExplainInfo(scala.Function1[String, Unit], SparkPlan) + qe.getClass.getDeclaredMethod( + "extendedExplainInfo", + classOf[String => Unit], + classOf[SparkPlan]) + } catch { + case _: NoSuchMethodException | _: SecurityException => return false + } + true + } + + // injectQueryStageOptimizerRule available since Spark 3.5 + def injectQueryStageOptimizerRules( + extensions: SparkSessionExtensions, + rule: SparkSession => Rule[SparkPlan]): Unit = { + extensions.injectQueryStageOptimizerRule(rule) + } +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala index e68c0cb3ec..3b416eea4d 100644 --- a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala @@ -19,8 +19,10 @@ package org.apache.comet.shims +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.expressions.aggregate.Aggregation -import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf @@ -30,4 +32,10 @@ trait ShimCometSparkSessionExtensions { protected def supportsExtendedExplainInfo(qe: QueryExecution): Boolean = true protected val EXTENDED_EXPLAIN_PROVIDERS_KEY = SQLConf.EXTENDED_EXPLAIN_PROVIDERS.key + + def injectQueryStageOptimizerRules( + extensions: SparkSessionExtensions, + rule: SparkSession => Rule[SparkPlan]): Unit = { + extensions.injectQueryStageOptimizerRule(rule) + } } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 93749185ef..9124f5ab07 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -2657,6 +2657,94 @@ class CometExecSuite extends CometTestBase { } } + test("DPP broadcast exchange reuse with AQE") { + // This test verifies that DPP works correctly when AQE is enabled. + // The fix involves deferring BroadcastHashJoinExec transformation in CometExecRule + // to allow PlanAdaptiveDynamicPruningFilters to find the join and create DPP, + // then CometBroadcastJoinRule transforms it afterward. + val factData = Seq[(Int, Int, Int, Int)]( + (1030, 3, 2, 10), + (1040, 3, 2, 50), + (1050, 3, 2, 50), + (1060, 3, 2, 50)) + + val storeData = Seq[(Int, String, String)]( + (1, "North-Holland", "NL"), + (2, "South-Holland", "NL"), + (3, "Bavaria", "DE")) + + withTable("fact_stats_aqe", "dim_stats_aqe") { + factData + .toDF("date_id", "store_id", "product_id", "units_sold") + .write + .partitionBy("store_id") + .format("parquet") + .saveAsTable("fact_stats_aqe") + + storeData + .toDF("store_id", "state_province", "country") + .write + .format("parquet") + .saveAsTable("dim_stats_aqe") + + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "true") { + + val df = sql(""" + |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats_aqe f + |JOIN dim_stats_aqe s + |ON f.store_id = s.store_id WHERE s.country = 'DE' + """.stripMargin) + + df.collect() + + val plan = df.queryExecution.executedPlan + println(s"DEBUG: executedPlan class: ${plan.getClass.getSimpleName}") + + // For AQE, get the actual executed plan + val actualPlan = plan match { + case aqe: AdaptiveSparkPlanExec => + println(s"DEBUG: AQE detected, getting executedPlan") + aqe.executedPlan + case other => other + } + println(s"DEBUG: actualPlan:\n$actualPlan") + + // Check that DPP is triggered - SubqueryBroadcastExec should be present + // Before the fix, PlanAdaptiveDynamicPruningFilters couldn't find BroadcastHashJoinExec + // (because Comet transformed it to CometBroadcastHashJoinExec first), so DPP was disabled + val subqueryBroadcasts = actualPlan.collectWithSubqueries { + case s: SubqueryBroadcastExec => + s + } + println(s"DEBUG: subqueryBroadcasts found: ${subqueryBroadcasts.size}") + assert( + subqueryBroadcasts.nonEmpty, + s"Expected SubqueryBroadcastExec for DPP with AQE but found none:\n$actualPlan") + + // Verify the join was transformed to Comet + val cometBroadcastJoins = actualPlan.collect { case j: CometBroadcastHashJoinExec => + j + } + val sparkBroadcastJoins = actualPlan.collect { case j: BroadcastHashJoinExec => + j + } + println(s"DEBUG: cometBroadcastJoins found: ${cometBroadcastJoins.size}") + println(s"DEBUG: sparkBroadcastJoins found: ${sparkBroadcastJoins.size}") + assert( + cometBroadcastJoins.nonEmpty, + s"Expected CometBroadcastHashJoinExec but found none:\n$actualPlan") + + checkSparkAnswer(df) + } + } + } + } case class BucketedTableTestSpec( bucketSpec: Option[BucketSpec], From 7e55091d9eecdf220760f9b4417e1b060f363152 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Feb 2026 15:06:19 -0500 Subject: [PATCH 38/38] try to do a deferred rule for aqe dpp --- .../scala/org/apache/comet/rules/CometBroadcastJoinRule.scala | 2 +- spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometBroadcastJoinRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometBroadcastJoinRule.scala index 0fd2164830..18397eb57e 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometBroadcastJoinRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometBroadcastJoinRule.scala @@ -137,7 +137,7 @@ case class CometBroadcastJoinRule(session: SparkSession) extends Rule[SparkPlan] Some(actualChild) case n: CometNativeExec => Some(n) - case other => + case _ => None } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 9124f5ab07..17a96e4809 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -2709,7 +2709,7 @@ class CometExecSuite extends CometTestBase { // For AQE, get the actual executed plan val actualPlan = plan match { case aqe: AdaptiveSparkPlanExec => - println(s"DEBUG: AQE detected, getting executedPlan") + println("DEBUG: AQE detected, getting executedPlan") aqe.executedPlan case other => other }